generated from streamnative/pulsar-io-template
-
Notifications
You must be signed in to change notification settings - Fork 25
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: Support new batch model BLEND/PARTITIONED (#1092)
* Revert "Add default value for the partitionType to `partition` (#863)" This reverts commit ba24f85. * Revert "[Proposal-1] Support Partitioner Refactoring (#845)" This reverts commit a8d9f1d. * feat: Add new batch model (cherry picked from commit 4b94112)
- Loading branch information
Showing
48 changed files
with
1,599 additions
and
1,449 deletions.
There are no files selected for viewing
This file was deleted.
Oops, something went wrong.
Large diffs are not rendered by default.
Oops, something went wrong.
Large diffs are not rendered by default.
Oops, something went wrong.
This file was deleted.
Oops, something went wrong.
Binary file not shown.
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
103 changes: 103 additions & 0 deletions
103
src/main/java/org/apache/pulsar/io/jcloud/batch/BatchContainer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/** | ||
* 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.pulsar.io.jcloud.batch; | ||
|
||
import com.google.common.collect.Lists; | ||
import java.util.List; | ||
import java.util.concurrent.ArrayBlockingQueue; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
import org.apache.pulsar.client.api.schema.GenericRecord; | ||
import org.apache.pulsar.functions.api.Record; | ||
|
||
/** | ||
* BatchContainer is used to store and manage batches of records. | ||
* It keeps track of the current batch size and bytes, and checks | ||
* if the batch needs to be flushed based on the max batch size, bytes, and time. | ||
*/ | ||
public class BatchContainer { | ||
|
||
private final long maxBatchSize; | ||
private final long maxBatchBytes; | ||
private final long maxBatchTimeMs; | ||
private final AtomicLong currentBatchSize = new AtomicLong(0L); | ||
private final AtomicLong currentBatchBytes = new AtomicLong(0L); | ||
private final ArrayBlockingQueue<Record<GenericRecord>> pendingFlushQueue; | ||
private volatile long lastPoolRecordsTime; | ||
|
||
public BatchContainer(long maxBatchSize, long maxBatchBytes, long maxBatchTimeMs, int maxPendingQueueSize) { | ||
this.maxBatchSize = maxBatchSize; | ||
this.maxBatchBytes = maxBatchBytes; | ||
this.maxBatchTimeMs = maxBatchTimeMs; | ||
this.pendingFlushQueue = new ArrayBlockingQueue<>(maxPendingQueueSize); | ||
this.lastPoolRecordsTime = System.currentTimeMillis(); | ||
} | ||
|
||
public void add(Record<GenericRecord> record) throws InterruptedException { | ||
pendingFlushQueue.put(record); | ||
updateCurrentBatchSize(1); | ||
updateCurrentBatchBytes(record.getMessage().get().size()); | ||
} | ||
|
||
public long getCurrentBatchSize() { | ||
return currentBatchSize.get(); | ||
} | ||
|
||
public long getCurrentBatchBytes() { | ||
return currentBatchBytes.get(); | ||
} | ||
|
||
public void updateCurrentBatchSize(long delta) { | ||
currentBatchSize.addAndGet(delta); | ||
} | ||
|
||
public void updateCurrentBatchBytes(long delta) { | ||
currentBatchBytes.addAndGet(delta); | ||
} | ||
|
||
public boolean isEmpty() { | ||
return pendingFlushQueue.isEmpty(); | ||
} | ||
|
||
public boolean needFlush() { | ||
long currentTime = System.currentTimeMillis(); | ||
return currentBatchSize.get() >= maxBatchSize | ||
|| currentBatchBytes.get() >= maxBatchBytes | ||
|| (currentTime - lastPoolRecordsTime) >= maxBatchTimeMs; | ||
} | ||
|
||
public List<Record<GenericRecord>> pollNeedFlushRecords() { | ||
final List<Record<GenericRecord>> needFlushRecords = Lists.newArrayList(); | ||
long recordsToInsertBytes = 0; | ||
while (!pendingFlushQueue.isEmpty() && needFlushRecords.size() < maxBatchSize | ||
&& recordsToInsertBytes < maxBatchBytes) { | ||
Record<GenericRecord> r = pendingFlushQueue.poll(); | ||
if (r != null) { | ||
if (r.getMessage().isPresent()) { | ||
long recordBytes = r.getMessage().get().size(); | ||
recordsToInsertBytes += recordBytes; | ||
} | ||
needFlushRecords.add(r); | ||
} | ||
} | ||
updateCurrentBatchBytes(-1 * recordsToInsertBytes); | ||
updateCurrentBatchSize(-1 * needFlushRecords.size()); | ||
lastPoolRecordsTime = System.currentTimeMillis(); | ||
return needFlushRecords; | ||
} | ||
} |
Oops, something went wrong.