-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/63230/#review189373
-----------------------------------------------------------




ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPool.java
Lines 120 (patched)
<https://reviews.apache.org/r/63230/#comment266424>

    Alternatively, since you are not looking at return value anyway. You can 
use CompletableFuture.allOf() which will returns one combined CompletableFuture 
which can be used to block.



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WmTezSession.java
Lines 83 (patched)
<https://reviews.apache.org/r/63230/#comment266426>

    This could be doable without Guava. With JDK8 
CompletableFuture.acceptEither.



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 264 (patched)
<https://reviews.apache.org/r/63230/#comment266430>

    I don't see value being used anywhere. Use Guava's Set.newIdentityHashSet() 
instead?



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Line 234 (original), 292 (patched)
<https://reviews.apache.org/r/63230/#comment266431>

    same here. awaitUninterruptibly()



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 332 (patched)
<https://reviews.apache.org/r/63230/#comment266446>

    cosmetic nit: 
    () -> {} lambda for runnable



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 363 (patched)
<https://reviews.apache.org/r/63230/#comment266432>

    nit: improve comment to say reclaim/rebalance what?



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 491 (patched)
<https://reviews.apache.org/r/63230/#comment266449>

    When will init be cancelled?
    
    If admin makes changes to pool, the event gets propagated and start with 
init state. Now if admin makes another change immediately (before the previous 
one completes), would that cancel the previous init?
    
    Is applying EventState synchronous?



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 503 (patched)
<https://reviews.apache.org/r/63230/#comment266457>

    Assuming this happens when there is no capacity in cluster for spinning up 
new sessios (or could be other reasons)? Is there a way to determine why this 
failed and throw error if not restartable (may be some permission issue)?



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1138 (patched)
<https://reviews.apache.org/r/63230/#comment266462>

    Create follow jira to handle this case? Killing all seems like overkill :P
    
    We could kill most recently submitted queries that will satisfy delta. Or 
make it optional to the user to pick the policy they want (pluggable).



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1223 (patched)
<https://reviews.apache.org/r/63230/#comment266433>

    nit: all CAPS for enum



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1262 (patched)
<https://reviews.apache.org/r/63230/#comment266436>

    is this a recoverable state. If not should session be set to null here?



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1270 (patched)
<https://reviews.apache.org/r/63230/#comment266434>

    Why handle unlock conditionally? handleSession* methods seems to be called 
only from here, so might as well unconditionally unlock in finally.



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1289 (patched)
<https://reviews.apache.org/r/63230/#comment266440>

    Both American and British English in one line. Cancel[l]ed :)



ql/src/java/org/apache/hadoop/hive/ql/exec/tez/WorkloadManager.java
Lines 1312 (patched)
<https://reviews.apache.org/r/63230/#comment266441>

    these two lines seems to be repeating? may be encapsulate inside a 
session.clear() or session.reset() method.


- 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
> 
>

Reply via email to