[ https://issues.apache.org/jira/browse/HIVE-24037?focusedWorklogId=595419&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-595419 ]
ASF GitHub Bot logged work on HIVE-24037: ----------------------------------------- Author: ASF GitHub Bot Created on: 12/May/21 15:38 Start Date: 12/May/21 15:38 Worklog Time Spent: 10m Work Description: pgaref commented on a change in pull request #2004: URL: https://github.com/apache/hive/pull/2004#discussion_r631157229 ########## 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(); + } Review comment: ets use a init method for these lines ########## 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(); + } Review comment: Lets use a init method for these lines -- 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 Issue Time Tracking ------------------- Worklog Id: (was: 595419) Time Spent: 1h 40m (was: 1.5h) > Parallelize hash table constructions in map joins > ------------------------------------------------- > > Key: HIVE-24037 > URL: https://issues.apache.org/jira/browse/HIVE-24037 > Project: Hive > Issue Type: Improvement > Reporter: Ramesh Kumar Thangarajan > Assignee: Ramesh Kumar Thangarajan > Priority: Major > Labels: pull-request-available > Time Spent: 1h 40m > Remaining Estimate: 0h > > Parallelize hash table constructions in map joins -- This message was sent by Atlassian Jira (v8.3.4#803005)