Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,18 @@ public void writeRecord(String database, String table, byte[] record) {
currentCacheBytes.addAndGet(bytes);
getLock(bufferKey).readLock().unlock();

if (flushQueue.size() < executionOptions.getFlushQueueSize()
&& (buffer.getBufferSizeBytes() >= executionOptions.getBufferFlushMaxBytes()
|| buffer.getNumOfRecords() >= executionOptions.getBufferFlushMaxRows())) {
boolean flush = bufferFullFlush(bufferKey);
LOG.info("trigger flush by buffer full, flush: {}", flush);
} else if (buffer.getBufferSizeBytes() >= STREAM_LOAD_MAX_BYTES
|| buffer.getNumOfRecords() >= STREAM_LOAD_MAX_ROWS) {
// The buffer capacity exceeds the stream load limit, flush
boolean flush = bufferFullFlush(bufferKey);
LOG.info("trigger flush by buffer exceeding the limit, flush: {}", flush);
Comment on lines +205 to +214
}

if (currentCacheBytes.get() > maxBlockedBytes) {
lock.lock();
try {
Expand All @@ -220,20 +232,6 @@ public void writeRecord(String database, String table, byte[] record) {
lock.unlock();
}
}

// queue has space, flush according to the bufferMaxRows/bufferMaxBytes
if (flushQueue.size() < executionOptions.getFlushQueueSize()
&& (buffer.getBufferSizeBytes() >= executionOptions.getBufferFlushMaxBytes()
|| buffer.getNumOfRecords() >= executionOptions.getBufferFlushMaxRows())) {
boolean flush = bufferFullFlush(bufferKey);
LOG.info("trigger flush by buffer full, flush: {}", flush);

} else if (buffer.getBufferSizeBytes() >= STREAM_LOAD_MAX_BYTES
|| buffer.getNumOfRecords() >= STREAM_LOAD_MAX_ROWS) {
// The buffer capacity exceeds the stream load limit, flush
boolean flush = bufferFullFlush(bufferKey);
LOG.info("trigger flush by buffer exceeding the limit, flush: {}", flush);
}
}

public boolean bufferFullFlush(String bufferKey) {
Expand Down Expand Up @@ -499,7 +497,7 @@ public void load(String label, BatchRecordBuffer buffer) throws IOException {
OBJECT_MAPPER.readValue(loadResult, RespContent.class);
if (DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) {
long cacheByteBeforeFlush =
currentCacheBytes.getAndAdd(-respContent.getLoadBytes());
currentCacheBytes.getAndAdd(-buffer.getBufferSizeBytes());
Comment on lines 499 to +500
LOG.info(
"load success, cacheBeforeFlushBytes: {}, currentCacheBytes : {}",
cacheByteBeforeFlush,
Expand Down
Loading