Skip to content

Commit

Permalink
[Improve] Fix the problem caused by very small interval in batch mode (
Browse files Browse the repository at this point in the history
  • Loading branch information
JNSimba authored Aug 15, 2024
1 parent 63b1290 commit 411020a
Show file tree
Hide file tree
Showing 16 changed files with 793 additions and 121 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,8 @@ public class DorisExecutionOptions implements Serializable {
private static final int DEFAULT_BUFFER_COUNT = 3;
// batch flush
private static final int DEFAULT_FLUSH_QUEUE_SIZE = 2;
private static final int DEFAULT_BUFFER_FLUSH_MAX_ROWS = 50000;
private static final int DEFAULT_BUFFER_FLUSH_MAX_BYTES = 10 * 1024 * 1024;
private static final int DEFAULT_BUFFER_FLUSH_MAX_ROWS = 500000;
private static final int DEFAULT_BUFFER_FLUSH_MAX_BYTES = 100 * 1024 * 1024;
private static final long DEFAULT_BUFFER_FLUSH_INTERVAL_MS = 10 * 1000;
private final int checkInterval;
private final int maxRetries;
Expand Down Expand Up @@ -358,9 +358,6 @@ public Builder setFlushQueueSize(int flushQueueSize) {
}

public Builder setBufferFlushIntervalMs(long bufferFlushIntervalMs) {
Preconditions.checkState(
bufferFlushIntervalMs >= 1000,
"bufferFlushIntervalMs must be greater than or equal to 1 second");
this.bufferFlushIntervalMs = bufferFlushIntervalMs;
return this;
}
Expand Down Expand Up @@ -397,6 +394,19 @@ public DorisExecutionOptions build() {
&& JSON.equals(streamLoadProp.getProperty(FORMAT_KEY))) {
streamLoadProp.put(READ_JSON_BY_LINE, true);
}

Preconditions.checkArgument(
bufferFlushIntervalMs >= 1000,
"bufferFlushIntervalMs must be greater than or equal to 1 second");

Preconditions.checkArgument(
bufferFlushMaxRows >= 10000,
"bufferFlushMaxRows must be greater than or equal to 10000");

Preconditions.checkArgument(
bufferFlushMaxBytes >= 10485760,
"bufferFlushMaxBytes must be greater than or equal to 10485760(10MB)");

return new DorisExecutionOptions(
checkInterval,
maxRetries,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.flink.sink.batch;

import org.apache.http.entity.AbstractHttpEntity;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.List;

public class BatchBufferHttpEntity extends AbstractHttpEntity {

private static final Logger LOG = LoggerFactory.getLogger(BatchBufferHttpEntity.class);
protected static final int OUTPUT_BUFFER_SIZE = 4096;
private final List<byte[]> buffer;
private final long contentLength;

public BatchBufferHttpEntity(BatchRecordBuffer recordBuffer) {
this.buffer = recordBuffer.getBuffer();
this.contentLength = recordBuffer.getBufferSizeBytes();
}

@Override
public boolean isRepeatable() {
return true;
}

@Override
public boolean isChunked() {
return false;
}

@Override
public long getContentLength() {
return contentLength;
}

@Override
public InputStream getContent() {
return new BatchBufferStream(buffer);
}

@Override
public void writeTo(OutputStream outStream) throws IOException {
try (InputStream inStream = new BatchBufferStream(buffer)) {
final byte[] buffer = new byte[OUTPUT_BUFFER_SIZE];
int readLen;
while ((readLen = inStream.read(buffer)) != -1) {
outStream.write(buffer, 0, readLen);
}
}
}

@Override
public boolean isStreaming() {
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.flink.sink.batch;

import java.io.IOException;
import java.io.InputStream;
import java.util.Iterator;
import java.util.List;

public class BatchBufferStream extends InputStream {
private final Iterator<byte[]> iterator;
private byte[] currentRow;
private int currentPos;

public BatchBufferStream(List<byte[]> buffer) {
this.iterator = buffer.iterator();
}

@Override
public int read() throws IOException {
return 0;
}

@Override
public int read(byte[] buf) throws IOException {
return read(buf, 0, buf.length);
}

@Override
public int read(byte[] buf, int off, int len) throws IOException {
if (!iterator.hasNext() && currentRow == null) {
return -1;
}

byte[] item = currentRow;
int pos = currentPos;
int readBytes = 0;
while (readBytes < len && (item != null || iterator.hasNext())) {
if (item == null) {
item = iterator.next();
pos = 0;
}

int size = Math.min(len - readBytes, item.length - pos);
System.arraycopy(item, pos, buf, off + readBytes, size);
readBytes += size;
pos += size;

if (pos == item.length) {
item = null;
pos = 0;
}
}
currentRow = item;
currentPos = pos;
return readBytes;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,80 +17,52 @@

package org.apache.doris.flink.sink.batch;

import org.apache.flink.annotation.VisibleForTesting;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.nio.ByteBuffer;
import java.util.LinkedList;

/** buffer to queue. */
public class BatchRecordBuffer {
private static final Logger LOG = LoggerFactory.getLogger(BatchRecordBuffer.class);
public static final String LINE_SEPARATOR = "\n";
private String labelName;
private ByteBuffer buffer;
private LinkedList<byte[]> buffer;
private byte[] lineDelimiter;
private int numOfRecords = 0;
private int bufferSizeBytes = 0;
private long bufferSizeBytes = 0;
private boolean loadBatchFirstRecord = true;
private String database;
private String table;
private final long createTime = System.currentTimeMillis();
private long retainTime = 0;

public BatchRecordBuffer() {}

public BatchRecordBuffer(byte[] lineDelimiter, int bufferSize) {
super();
this.lineDelimiter = lineDelimiter;
this.buffer = ByteBuffer.allocate(bufferSize);
public BatchRecordBuffer() {
this.buffer = new LinkedList<>();
}

public BatchRecordBuffer(String database, String table, byte[] lineDelimiter, int bufferSize) {
public BatchRecordBuffer(String database, String table, byte[] lineDelimiter, long retainTime) {
super();
this.database = database;
this.table = table;
this.lineDelimiter = lineDelimiter;
this.buffer = ByteBuffer.allocate(bufferSize);
this.buffer = new LinkedList<>();
this.retainTime = retainTime;
}

public void insert(byte[] record) {
ensureCapacity(record.length);
public int insert(byte[] record) {
int recordSize = record.length;
if (loadBatchFirstRecord) {
loadBatchFirstRecord = false;
} else if (lineDelimiter != null) {
this.buffer.put(this.lineDelimiter);
this.buffer.add(this.lineDelimiter);
setBufferSizeBytes(this.bufferSizeBytes + this.lineDelimiter.length);
recordSize += this.lineDelimiter.length;
}
this.buffer.put(record);
setNumOfRecords(getNumOfRecords() + 1);
setBufferSizeBytes(getBufferSizeBytes() + record.length);
}

@VisibleForTesting
public void ensureCapacity(int length) {
int lineDelimiterSize = this.lineDelimiter == null ? 0 : this.lineDelimiter.length;
if (buffer.remaining() - lineDelimiterSize >= length) {
return;
}
int currentRemain = buffer.remaining();
int currentCapacity = buffer.capacity();
// add lineDelimiter length
int needed = length - buffer.remaining() + lineDelimiterSize;
// grow at least 1MB
long grow = Math.max(needed, 1024 * 1024);
// grow at least 50% of the current size
grow = Math.max(buffer.capacity() / 2, grow);
int newCapacity = (int) Math.min(Integer.MAX_VALUE, buffer.capacity() + grow);
ByteBuffer tmp = ByteBuffer.allocate(newCapacity);
buffer.flip();
tmp.put(buffer);
buffer.clear();
buffer = tmp;
LOG.info(
"record length {},buffer remain {} ,grow capacity {} to {}",
length,
currentRemain,
currentCapacity,
newCapacity);
this.buffer.add(record);
setNumOfRecords(this.numOfRecords + 1);
setBufferSizeBytes(this.bufferSizeBytes + record.length);
return recordSize;
}

public String getLabelName() {
Expand All @@ -106,13 +78,6 @@ public boolean isEmpty() {
return numOfRecords == 0;
}

public ByteBuffer getData() {
// change mode
buffer.flip();
LOG.debug("flush buffer: {} records, {} bytes", getNumOfRecords(), getBufferSizeBytes());
return buffer;
}

public void clear() {
this.buffer.clear();
this.numOfRecords = 0;
Expand All @@ -121,7 +86,7 @@ public void clear() {
this.loadBatchFirstRecord = true;
}

public ByteBuffer getBuffer() {
public LinkedList<byte[]> getBuffer() {
return buffer;
}

Expand All @@ -131,7 +96,7 @@ public int getNumOfRecords() {
}

/** @return Buffer size in bytes */
public int getBufferSizeBytes() {
public long getBufferSizeBytes() {
return bufferSizeBytes;
}

Expand All @@ -141,7 +106,7 @@ public void setNumOfRecords(int numOfRecords) {
}

/** @param bufferSizeBytes Updates sum of size of records present in this buffer (Bytes) */
public void setBufferSizeBytes(int bufferSizeBytes) {
public void setBufferSizeBytes(long bufferSizeBytes) {
this.bufferSizeBytes = bufferSizeBytes;
}

Expand All @@ -160,4 +125,22 @@ public String getTable() {
public void setTable(String table) {
this.table = table;
}

public String getTableIdentifier() {
if (database != null && table != null) {
return database + "." + table;
}
return null;
}

public byte[] getLineDelimiter() {
return lineDelimiter;
}

public boolean shouldFlush() {
// When the buffer create time is later than the first interval trigger,
// the write will not be triggered in the next interval,
// so multiply it by 1.5 to trigger it as early as possible.
return (System.currentTimeMillis() - createTime) * 1.5 > retainTime;
}
}
Loading

0 comments on commit 411020a

Please sign in to comment.