drill-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] sohami commented on a change in pull request #1459: DRILL-6731: Move the BFs aggregating work from the Foreman to the RuntimeFi…
Date Thu, 27 Sep 2018 20:39:05 GMT
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_r221069294
 
 

 ##########
 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:
   Thanks for explanation. So for single minor fragment left deep tree case as shown below
there will be only one `RuntimeFilterOperator` (RTF) inserted above left most Scan. Now when
`next() `is called then first upper HJ build side will be evaluated which will create BloomFilter
and send to RTF operator. Later `next()` will be called on probe side of first upper HJ which
will again result in calling `next() `on build side of lower hash join. When build side of
lower join is completed then it will again send its BloomFilter to RTF operator. Considering
bloom filter received is from 2 different HJ operators (hence different `srcHashJoinOpId`)
it will discard the first one and keep the second one. I guess this is done because join condition
column can be different in different HashJoin ?
   
   ```
         HJ
        /  \
      HJ    Scan
     /  \
   Scan  Scan
   ```
   
   ```
         HJ
        /  \
      HJ    Scan
     /  \
   RTF  Scan
     |
   Scan
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message