pgaref commented on a change in pull request #2004:
URL: https://github.com/apache/hive/pull/2004#discussion_r631163313



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTableLoader.java
##########
@@ -141,35 +280,64 @@ public void load(MapJoinTableContainer[] mapJoinTables,
         long keyCount = Math.max(estKeyCount, inputRecords);
 
         VectorMapJoinFastTableContainer vectorMapJoinFastTableContainer =
-                new VectorMapJoinFastTableContainer(desc, hconf, keyCount);
+                new VectorMapJoinFastTableContainer(desc, hconf, keyCount, 
numThreads);
 
         LOG.info("Loading hash table for input: {} cacheKey: {} 
tableContainer: {} smallTablePos: {} " +
                 "estKeyCount : {} keyCount : {}", inputName, cacheKey,
                 vectorMapJoinFastTableContainer.getClass().getSimpleName(), 
pos, estKeyCount, keyCount);
 
         vectorMapJoinFastTableContainer.setSerde(null, null); // No SerDes 
here.
+        ExecutorService executorService = 
Executors.newFixedThreadPool(numThreads);
+        BlockingQueue<QueueElementBatch>[] sharedQ = new 
BlockingQueue[numThreads];
+        for(int i = 0; i < numThreads; ++i) {
+          sharedQ[i] = new LinkedBlockingQueue<>();
+        }
+        QueueElementBatch[] batches = new QueueElementBatch[numThreads];
+        for (int i = 0; i < numThreads; ++i) {
+          batches[i] = new QueueElementBatch();
+        }
+        //start the threads
+        drain(vectorMapJoinFastTableContainer, doMemCheck, inputName, 
memoryMonitorInfo,
+            effectiveThreshold, executorService, sharedQ);
         long startTime = System.currentTimeMillis();
         while (kvReader.next()) {
-          
vectorMapJoinFastTableContainer.putRow((BytesWritable)kvReader.getCurrentKey(),
-              (BytesWritable)kvReader.getCurrentValue());
+          BytesWritable currentKey = (BytesWritable) kvReader.getCurrentKey();
+          BytesWritable currentValue = (BytesWritable) 
kvReader.getCurrentValue();
+          long key = 
vectorMapJoinFastTableContainer.deserializeToKey(currentKey);
+          long hashCode = 
vectorMapJoinFastTableContainer.calculateLongHashCode(key, currentKey);
+          int partitionId = (int) ((numThreads - 1) & hashCode);
           numEntries++;
-          if (doMemCheck && (numEntries % 
memoryMonitorInfo.getMemoryCheckInterval() == 0)) {
-              final long estMemUsage = 
vectorMapJoinFastTableContainer.getEstimatedMemorySize();
-              if (estMemUsage > effectiveThreshold) {
-                String msg = "Hash table loading exceeded memory limits for 
input: " + inputName +
-                  " numEntries: " + numEntries + " estimatedMemoryUsage: " + 
estMemUsage +
-                  " effectiveThreshold: " + effectiveThreshold + " 
memoryMonitorInfo: " + memoryMonitorInfo;
-                LOG.error(msg);
-                throw new MapJoinMemoryExhaustionError(msg);
-              } else {
-                if (LOG.isInfoEnabled()) {
-                  LOG.info("Checking hash table loader memory usage for input: 
{} numEntries: {} " +
-                      "estimatedMemoryUsage: {} effectiveThreshold: {}", 
inputName, numEntries, estMemUsage,
-                    effectiveThreshold);
-                }
-              }
+          // call getBytes as copy is called later
+          byte[] valueBytes = currentValue.copyBytes();
+          int valueLength = currentValue.getLength();
+          byte[] keyBytes = currentKey.copyBytes();
+          int keyLength = currentKey.getLength();
+          HashTableElement h = new HashTableElement(keyBytes, keyLength, 
valueBytes, valueLength, key, hashCode);
+          if (batches[partitionId].addElement(h)) {
+              sharedQ[partitionId].add(batches[partitionId]);
+              batches[partitionId] = new QueueElementBatch();
           }
         }
+
+        LOG.info("Finished loading the queue for input: {} endTime : {}", 
inputName, System.currentTimeMillis());
+
+        // Add sentinel at the end of queue
+        for (int i=0; i<4; ++i) {
+          // add sentinel to the q not the batch
+          sharedQ[i].add(batches[i]);
+          sharedQ[i].add(sentinel);
+        }
+
+        executorService.shutdown();
+        try {
+          executorService.awaitTermination(Long.MAX_VALUE, 
TimeUnit.NANOSECONDS);
+        } catch (InterruptedException e) {

Review comment:
       handle exception along with others at the end?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org
For additional commands, e-mail: gitbox-h...@hive.apache.org

Reply via email to