----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/63230/#review189249 -----------------------------------------------------------
ql/src/java/org/apache/hadoop/hive/ql/exec/tez/QueryAllocationManager.java Lines 29 (patched) <https://reviews.apache.org/r/63230/#comment266246> nit: negative or null? ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Line 88 (original), 100 (patched) <https://reviews.apache.org/r/63230/#comment266247> nit: <=1 ? ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Line 98 (original), 110 (patched) <https://reviews.apache.org/r/63230/#comment266252> would be cleaner with executor service + thread factory to name the thread based on idx or session name instead of using FutureTask directly. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Line 102 (original), 114 (patched) <https://reviews.apache.org/r/63230/#comment266250> why this special casing here? why are invoking threadTasks[0].run() only for i== 0 and not others ? ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 120 (patched) <https://reviews.apache.org/r/63230/#comment266253> Better to use ListenableFuture instead of potential blocking here during future.get() ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 130 (patched) <https://reviews.apache.org/r/63230/#comment266255> use notEmpty.awaitUninterruptibly() ? won't wake up for spurious interrupts. no need to loop with that. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 196 (patched) <https://reviews.apache.org/r/63230/#comment266257> could you add some comments here. can't understand what is going on in this logic. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 257 (patched) <https://reviews.apache.org/r/63230/#comment266256> unused variables? ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 335 (patched) <https://reviews.apache.org/r/63230/#comment266258> use sequence number from znode for now? assuming restart will generate new znode and sequence number. else UUID in service instance. Can you add logging here to record the create/remove/update events for debugging in case if this issue happens. Please file jira for tracking this. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 347 (patched) <https://reviews.apache.org/r/63230/#comment266259> make this dummy true future a static constant. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 372 (patched) <https://reviews.apache.org/r/63230/#comment266261> this entire block looks repeated here and for initial sessions. put this in a separate method instead? also can use single thread pool/executor service for this as mentioned in earlier comments. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 401 (patched) <https://reviews.apache.org/r/63230/#comment266263> when, how, who will drain sessions in toClose? I don't see it being drained and closed anywhere in this patch. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java Lines 434 (patched) <https://reviews.apache.org/r/63230/#comment266266> use CountDownLatch instead? before creating CreateSessionsRunnable we know how many sessions we want to create (initial or delta). We can set that to latch and these runnables can simply decrement it after session creation. Main thread can wait for it to become zero. Easier to read and few LOC. ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java Lines 179 (patched) <https://reviews.apache.org/r/63230/#comment266267> nit: use different name for timepool threads. - Prasanth_J On Oct. 24, 2017, 11:58 p.m., Sergey Shelukhin wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/63230/ > ----------------------------------------------------------- > > (Updated Oct. 24, 2017, 11:58 p.m.) > > > Review request for hive and Prasanth_J. > > > Repository: hive-git > > > Description > ------- > > see jira > > > Diffs > ----- > > common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 0b4abb824f > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/AmPluginNode.java 35d380c92b > > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/GuaranteedTasksAllocator.java > d978a25b14 > > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/LlapPluginEndpointClientImpl.java > 45c3e38dcc > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/QueryAllocationManager.java > a326db3ab0 > > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/SessionExpirationTracker.java > da93a3a791 > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java > b67c933b19 > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java > 9b4714f1d7 > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolSession.java > 613522357e > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java > 144816862d > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/UserPoolMapping.java > 81d6b859a6 > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WmTezSession.java 00501eef93 > ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java > d725e90475 > > ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LlapClusterStateForCompile.java > 209cf57a6a > ql/src/test/org/apache/hadoop/hive/ql/exec/tez/SampleTezSessionState.java > 59efd43be6 > ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestWorkloadManager.java > 258a865fef > > > Diff: https://reviews.apache.org/r/63230/diff/2/ > > > Testing > ------- > > > Thanks, > > Sergey Shelukhin > >