[
https://issues.apache.org/jira/browse/DRILL-6731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16629368#comment-16629368
]
ASF GitHub Bot commented on DRILL-6731:
---------------------------------------
sohami commented on a change in pull request #1459: DRILL-6731: Move the BFs
aggregating work from the Foreman to the RuntimeFi…
URL: https://github.com/apache/drill/pull/1459#discussion_r220256674
##########
File path:
exec/java-exec/src/main/java/org/apache/drill/exec/work/filter/RuntimeFilterSink.java
##########
@@ -36,25 +40,63 @@
private RuntimeFilterWritable aggregated = null;
- private Queue<RuntimeFilterWritable> rfQueue = new ConcurrentLinkedQueue<>();
+ private BlockingQueue<RuntimeFilterWritable> rfQueue = new
LinkedBlockingQueue<>();
private AtomicBoolean running = new AtomicBoolean(true);
+ private ReentrantLock aggregatedRFLock = new ReentrantLock();
+
+ private Thread asyncAggregateThread;
+
+ private BufferAllocator bufferAllocator;
+
+ private static final Logger logger =
LoggerFactory.getLogger(RuntimeFilterSink.class);
+
+
+ public RuntimeFilterSink(BufferAllocator bufferAllocator) {
+ this.bufferAllocator = bufferAllocator;
+ AsyncAggregateWorker asyncAggregateWorker = new AsyncAggregateWorker();
+ asyncAggregateThread = new
NamedThreadFactory("RFAggregating-").newThread(asyncAggregateWorker);
+ asyncAggregateThread.start();
+ }
+
public void aggregate(RuntimeFilterWritable runtimeFilterWritable) {
- rfQueue.add(runtimeFilterWritable);
- if (currentBookId.get() == 0) {
- AsyncAggregateWorker asyncAggregateWorker = new AsyncAggregateWorker();
- Thread asyncAggregateThread = new
NamedThreadFactory("RFAggregating-").newThread(asyncAggregateWorker);
- asyncAggregateThread.start();
+ if (running.get()) {
+ if (containOne()) {
+ boolean same = aggregated.same(runtimeFilterWritable);
+ if (!same) {
+ //This is to solve the only one fragment case that two
RuntimeFilterRecordBatchs
+ //share the same FragmentContext.
Review comment:
Can you please elaborate on this use case ? I didn't quite get it. Based on
my understanding for one fragment case the RuntimeFilter will be directly set
in the FragmentContext and will not be sent over wire. So we should not have
received it in first place ?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> JPPD:Move aggregating the BF from the Foreman to the RuntimeFilter
> ------------------------------------------------------------------
>
> Key: DRILL-6731
> URL: https://issues.apache.org/jira/browse/DRILL-6731
> Project: Apache Drill
> Issue Type: Improvement
> Components: Server
> Affects Versions: 1.15.0
> Reporter: weijie.tong
> Assignee: weijie.tong
> Priority: Major
>
> This PR is to move the BloomFilter aggregating work from the foreman to
> RuntimeFilter. Though this change, the RuntimeFilter can apply the incoming
> BF as soon as possible.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)