[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-21 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17041882#comment-17041882
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

asfgit commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981
 
 
   
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
>  Labels: ready-to-commit
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-16 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037992#comment-17037992
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on issue #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#issuecomment-586762991
 
 
   @ihuzenko, sorry for being unclear. I am suggesting that this PR focus its 
functional change on removing the `STOP` status and revising the `kill()` 
method. There are cosmetic changes in files visited during this work; changes 
which should be low-risk and, IMHO, don't justify their own PR. (That said, we 
could certainly pull out cosmetic changes to a separate PR; some reviewers 
prefer that approach.)
   
   You make good suggestions about additional refactoring and we should make 
those improvements. To limit risk, I would suggest we tackle those in a 
separate PR.
   
   Does this work for you? Anything else that I should fix in this PR?
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037737#comment-17037737
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on issue #1981: DRILL-7583: Remove STOP status from operator 
outcome
URL: https://github.com/apache/drill/pull/1981#issuecomment-586678486
 
 
   Hello @paul-rogers , 
   
   Sorry if I didn't understand you very well, but since I had seen about 78 
files changed I thought that adding a few more changes to the PR is not an 
issue. Although if you don't have time to apply suggestions, I won't force 
them, we can merge the PR, since I really like what is already done.
   
   Thanks, 
   Igor
   
   
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037726#comment-17037726
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379881384
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -479,34 +493,41 @@ private IterOutcome sniffNonEmptyBatch(IterOutcome curr, 
int inputIndex, RecordB
   curr = next(inputIndex, recordBatch);
 
   switch (curr) {
-case OK:
-  // We got a data batch
-  break;
-case NOT_YET:
-  // We need to try again
-  break;
-case EMIT:
-  throw new UnsupportedOperationException("We do not support " + EMIT);
-default:
-  // Other cases are termination conditions
-  return curr;
+  case OK:
+// We got a data batch
+break;
+  case NOT_YET:
+// We need to try again
+break;
+  case EMIT:
+throw new UnsupportedOperationException("We do not support " + EMIT);
+  default:
+// Other cases are termination conditions
+return curr;
   }
 }
   }
 
   /**
-   * Determines the memory calculator to use. If maxNumBatches is configured 
simple batch counting is used to spill. Otherwise
-   * memory calculations are used to determine when to spill.
+   * Determines the memory calculator to use. If maxNumBatches is configured
+   * simple batch counting is used to spill. Otherwise memory calculations are
+   * used to determine when to spill.
+   *
* @return The memory calculator to use.
*/
   public HashJoinMemoryCalculator getCalculatorImpl() {
 if (maxBatchesInMemory == 0) {
-  double safetyFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
-  double fragmentationFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
-  double hashTableDoublingFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
-  String hashTableCalculatorType = 
context.getOptions().getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
-
-  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor, hashTableDoublingFactor, hashTableCalculatorType, 
semiJoin);
+  double safetyFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
+  double fragmentationFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
+  double hashTableDoublingFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
+  String hashTableCalculatorType = context.getOptions()
+  .getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
+
+  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor,
+  hashTableDoublingFactor, hashTableCalculatorType, semiJoin);
 
 Review comment:
   I'm already working on some refactoring in the scope of my DRILL-4092 work. 
Since for the task I have to deal with operator execution very closely I'll do 
my best to leave code in a better state.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037577#comment-17037577
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on issue #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#issuecomment-586632824
 
 
   The Java 13 build failed with a UDF failure:
   
   ```
   TestDynamicUDFSupport.testReRegisterTheSameJarWithDifferentContent
   ```
   
   In past months, I found that some of the UDF tests where very flaky in my 
own environment. The Guava patcher may have fixed them as the failures 
disappeared around the time of that fix.
   
   In any event, this PR does not change any UDF-related code. Is this a Java 
13 specific failure?
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037561#comment-17037561
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848965
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -679,67 +715,91 @@ public IterOutcome innerNext() {
   }
 
   /**
-   * In case an upstream data is no longer needed, send a kill and flush any 
remaining batch
+   * In case an upstream data is no longer needed, send a kill and flush any
+   * remaining batch
*
-   * @param batch probe or build batch
-   * @param upstream which upstream
-   * @param isLeft is it the left or right
+   * @param batch
+   *  probe or build batch
+   * @param upstream
+   *  which upstream
+   * @param isLeft
+   *  is it the left or right
*/
-  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream, 
boolean isLeft) {
-batch.kill(true);
-while (upstream == IterOutcome.OK_NEW_SCHEMA || upstream == 
IterOutcome.OK) {
+  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream,
+  boolean isLeft) {
 
 Review comment:
   Another good suggestion for a refactoring pass.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037557#comment-17037557
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379847829
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -308,35 +324,28 @@ public boolean hasPartitionLimit() {
*/
   private final SpilledState spilledState = new 
SpilledState<>();
   private final HashJoinUpdater spilledStateUpdater = new HashJoinUpdater();
-  private HashJoinSpilledPartition spilledInners[]; // for the outer to find 
the partition
+  private HashJoinSpilledPartition spilledInners[]; // for the outer to find 
the
+// partition
 
   public enum Metric implements MetricDef {
-NUM_BUCKETS,
-NUM_ENTRIES,
-NUM_RESIZING,
-RESIZING_TIME_MS,
-NUM_PARTITIONS,
-SPILLED_PARTITIONS, // number of original partitions spilled to disk
-SPILL_MB, // Number of MB of data spilled to disk. This amount is 
first written,
-  // then later re-read. So, disk I/O is twice this amount.
-SPILL_CYCLE,   // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
-LEFT_INPUT_BATCH_COUNT,
-LEFT_AVG_INPUT_BATCH_BYTES,
-LEFT_AVG_INPUT_ROW_BYTES,
-LEFT_INPUT_RECORD_COUNT,
-RIGHT_INPUT_BATCH_COUNT,
-RIGHT_AVG_INPUT_BATCH_BYTES,
-RIGHT_AVG_INPUT_ROW_BYTES,
-RIGHT_INPUT_RECORD_COUNT,
-OUTPUT_BATCH_COUNT,
-AVG_OUTPUT_BATCH_BYTES,
-AVG_OUTPUT_ROW_BYTES,
-OUTPUT_RECORD_COUNT;
+NUM_BUCKETS, NUM_ENTRIES, NUM_RESIZING, RESIZING_TIME_MS, NUM_PARTITIONS,
+// number of original partitions spilled to disk
+SPILLED_PARTITIONS,
+SPILL_MB, // Number of MB of data spilled to disk. This amount is first
+  // written,
+  // then later re-read. So, disk I/O is twice this amount.
+SPILL_CYCLE, // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+LEFT_INPUT_BATCH_COUNT, LEFT_AVG_INPUT_BATCH_BYTES, 
LEFT_AVG_INPUT_ROW_BYTES,
+LEFT_INPUT_RECORD_COUNT, RIGHT_INPUT_BATCH_COUNT, 
RIGHT_AVG_INPUT_BATCH_BYTES,
+RIGHT_AVG_INPUT_ROW_BYTES, RIGHT_INPUT_RECORD_COUNT, OUTPUT_BATCH_COUNT,
+AVG_OUTPUT_BATCH_BYTES, AVG_OUTPUT_ROW_BYTES, OUTPUT_RECORD_COUNT;
 
 Review comment:
   Argh... An auto-format of a comment got loose and did the whole file. Fixed.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037562#comment-17037562
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379849068
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -1344,74 +1483,95 @@ private void updateStats() {
 stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
 stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
 stats.setLongStat(Metric.NUM_PARTITIONS, numPartitions);
-stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 
in case no spill
+stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in
+// case no
+// spill
 stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
   }
 
   /**
-   * Get the hash table iterator that is created for the build side of the 
hash join if
-   * this hash join was instantiated as a row-key join.
-   * @return hash table iterator or null if this hash join was not a row-key 
join or if it
-   * was a row-key join but the build has not yet completed.
+   * Get the hash table iterator that is created for the build side of the hash
+   * join if this hash join was instantiated as a row-key join.
+   *
+   * @return hash table iterator or null if this hash join was not a row-key
+   * join or if it was a row-key join but the build has not yet
+   * completed.
*/
   @Override
   public Pair nextRowKeyBatch() {
 if (buildComplete) {
-  // partition 0 because Row Key Join has only a single partition - no 
spilling
+  // partition 0 because Row Key Join has only a single partition - no
+  // spilling
   Pair pp = partitions[0].nextBatch();
   if (pp != null) {
 VectorWrapper vw = Iterables.get(pp.getLeft(), 0);
 ValueVector vv = vw.getValueVector();
 return Pair.of(vv, pp.getRight());
   }
-} else if(partitions == null && firstOutputBatch) { //if there is data 
coming to right(build) side in build Schema stage, use it.
+} else if (partitions == null && firstOutputBatch) { // if there is data
+ // coming to
+ // right(build) side 
in
+ // build Schema stage,
+ // use it.
   firstOutputBatch = false;
-  if ( right.getRecordCount() > 0 ) {
+  if (right.getRecordCount() > 0) {
 VectorWrapper vw = Iterables.get(right, 0);
 ValueVector vv = vw.getValueVector();
-return Pair.of(vv, right.getRecordCount()-1);
+return Pair.of(vv, right.getRecordCount() - 1);
   }
 }
 return null;
   }
 
-  @Override// implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public boolean hasRowKeyBatch() {
 return buildComplete;
   }
 
-  @Override   // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public BatchState getBatchState() {
 return state;
   }
 
-  @Override  // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public void setBatchState(BatchState newState) {
 state = newState;
   }
 
   @Override
-  public void killIncoming(boolean sendUpstream) {
+  protected void cancelIncoming() {
 wasKilled = true;
-probeBatch.kill(sendUpstream);
-buildBatch.kill(sendUpstream);
+probeBatch.cancel();
+buildBatch.cancel();
   }
 
   public void updateMetrics() {
-stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_BATCH_COUNT, 
batchMemoryManager.getNumIncomingBatches(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_BATCH_BYTES, 
batchMemoryManager.getAvgInputBatchSize(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_ROW_BYTES, 
batchMemoryManager.getAvgInputRowWidth(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_RECORD_COUNT, 
batchMemoryManager.getTotalInputRecords(LEFT_INDEX));
-
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_BATCH_COUNT, 
batchMemoryManager.getNumIncomingBatches(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_BATCH_BYTES, 
batchMemoryManager.getAvgInputBatchSize(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_ROW_BYTES, 
batchMemoryManager.getAvgInputRowWidth(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_RECORD_COUNT, 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037558#comment-17037558
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848754
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -386,52 +398,52 @@ protected void buildSchema() {
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstBuildBatch() {
-rightUpstream = prefetchFirstBatch(rightUpstream,
-  prefetchedBuild,
-  buildSideIsEmpty,
-  RIGHT_INDEX,
-  buildBatch,
-  () -> {
-batchMemoryManager.update(RIGHT_INDEX, 0, true);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
-batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
-getRecordBatchStatsContext());
-  });
+rightUpstream = prefetchFirstBatch(rightUpstream, prefetchedBuild,
+buildSideIsEmpty, RIGHT_INDEX, buildBatch, () -> {
+  batchMemoryManager.update(RIGHT_INDEX, 0, true);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
+  batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
+  getRecordBatchStatsContext());
+});
 
 Review comment:
   Good suggestion. For the most part, I tried not to tinker with the code here 
unless necessary. A few of the adjustments to remove STOP resulted in subtle 
errors that took hours to find. In general, this class (and several others) are 
trying to do too much: there are too many variables to think about. Better to 
split up responsibilities as I did years ago in the external sort. (For fun, go 
back and look at the original version.)
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037564#comment-17037564
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848926
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -479,34 +493,41 @@ private IterOutcome sniffNonEmptyBatch(IterOutcome curr, 
int inputIndex, RecordB
   curr = next(inputIndex, recordBatch);
 
   switch (curr) {
-case OK:
-  // We got a data batch
-  break;
-case NOT_YET:
-  // We need to try again
-  break;
-case EMIT:
-  throw new UnsupportedOperationException("We do not support " + EMIT);
-default:
-  // Other cases are termination conditions
-  return curr;
+  case OK:
+// We got a data batch
+break;
+  case NOT_YET:
+// We need to try again
+break;
+  case EMIT:
+throw new UnsupportedOperationException("We do not support " + EMIT);
+  default:
+// Other cases are termination conditions
+return curr;
   }
 }
   }
 
   /**
-   * Determines the memory calculator to use. If maxNumBatches is configured 
simple batch counting is used to spill. Otherwise
-   * memory calculations are used to determine when to spill.
+   * Determines the memory calculator to use. If maxNumBatches is configured
+   * simple batch counting is used to spill. Otherwise memory calculations are
+   * used to determine when to spill.
+   *
* @return The memory calculator to use.
*/
   public HashJoinMemoryCalculator getCalculatorImpl() {
 if (maxBatchesInMemory == 0) {
-  double safetyFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
-  double fragmentationFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
-  double hashTableDoublingFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
-  String hashTableCalculatorType = 
context.getOptions().getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
-
-  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor, hashTableDoublingFactor, hashTableCalculatorType, 
semiJoin);
+  double safetyFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
+  double fragmentationFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
+  double hashTableDoublingFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
+  String hashTableCalculatorType = context.getOptions()
+  .getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
+
+  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor,
+  hashTableDoublingFactor, hashTableCalculatorType, semiJoin);
 
 Review comment:
   Another good suggestion. After this PR is in, do you want to take a crack at 
fixing some of this stuff?
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037559#comment-17037559
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848798
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -386,52 +398,52 @@ protected void buildSchema() {
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstBuildBatch() {
-rightUpstream = prefetchFirstBatch(rightUpstream,
-  prefetchedBuild,
-  buildSideIsEmpty,
-  RIGHT_INDEX,
-  buildBatch,
-  () -> {
-batchMemoryManager.update(RIGHT_INDEX, 0, true);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
-batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
-getRecordBatchStatsContext());
-  });
+rightUpstream = prefetchFirstBatch(rightUpstream, prefetchedBuild,
+buildSideIsEmpty, RIGHT_INDEX, buildBatch, () -> {
+  batchMemoryManager.update(RIGHT_INDEX, 0, true);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
+  batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
+  getRecordBatchStatsContext());
+});
   }
 
   /**
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstProbeBatch() {
-leftUpstream =  prefetchFirstBatch(leftUpstream,
-  prefetchedProbe,
-  probeSideIsEmpty,
-  LEFT_INDEX,
-  probeBatch,
-  () -> {
-batchMemoryManager.update(LEFT_INDEX, 0);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
-batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
-getRecordBatchStatsContext());
-  });
+leftUpstream = prefetchFirstBatch(leftUpstream, prefetchedProbe,
+probeSideIsEmpty, LEFT_INDEX, probeBatch, () -> {
+  batchMemoryManager.update(LEFT_INDEX, 0);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
+  batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
+  getRecordBatchStatsContext());
+});
   }
 
   /**
-   * Used to fetch the first data holding batch from either the build or probe 
side.
-   * @param outcome The current upstream outcome for either the build or probe 
side.
-   * @param prefetched A flag indicating if we have already done a prefetch of 
the first data holding batch for the probe or build side.
-   * @param isEmpty A flag indicating if the probe or build side is empty.
-   * @param index The upstream index of the probe or build batch.
-   * @param batch The probe or build batch itself.
-   * @param memoryManagerUpdate A lambda function to execute the memory 
manager update for the probe or build batch.
-   * @return The current {@link 
org.apache.drill.exec.record.RecordBatch.IterOutcome}.
+   * Used to fetch the first data holding batch from either the build or probe
+   * side.
+   *
+   * @param outcome
+   *  The current upstream outcome for either the build or probe side.
+   * @param prefetched
+   *  A flag indicating if we have already done a prefetch of the first
+   *  data holding batch for the probe or build side.
+   * @param isEmpty
+   *  A flag indicating if the probe or build side is empty.
+   * @param index
+   *  The upstream index of the probe or build batch.
+   * @param batch
+   *  The probe or build batch itself.
+   * @param memoryManagerUpdate
+   *  A lambda function to execute the memory manager update for the
+   *  probe or build batch.
+   * @return The current
+   * {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}.
*/
   private IterOutcome prefetchFirstBatch(IterOutcome outcome,
- MutableBoolean prefetched,
- MutableBoolean isEmpty,
- int index,
- RecordBatch batch,
- Runnable memoryManagerUpdate) {
+  MutableBoolean prefetched, MutableBoolean isEmpty, int index,
 
 Review comment:
   Reverted, Another over-aggressive bit of help from the code formatter.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037566#comment-17037566
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379849340
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
 ##
 @@ -88,6 +88,7 @@ public void TestMultipleSendLocationBroadcastExchange() 
throws Exception {
 }
 b.release();
   }
+  // Nothing done with count?
 
 Review comment:
   Could be. I didn't spend time to work out what the test is doing; just 
noticed that this is a "it works if it does not crash" kind of test.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037560#comment-17037560
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848844
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -445,32 +457,34 @@ private IterOutcome prefetchFirstBatch(IterOutcome 
outcome,
   outcome = sniffNonEmptyBatch(outcome, index, batch);
 }
 
-isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE 
there is no data.
+isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE there
+   // is no data.
 
-if (outcome == IterOutcome.STOP) {
-  // We reached a termination state
-  state = BatchState.STOP;
-} else {
-  // Got our first batch(es)
-  if (spilledState.isFirstCycle()) {
-// Only collect stats for the first cycle
-memoryManagerUpdate.run();
-  }
-  state = BatchState.FIRST;
+// Got our first batch(es)
+if (spilledState.isFirstCycle()) {
+  // Only collect stats for the first cycle
+  memoryManagerUpdate.run();
 }
-
+state = BatchState.FIRST;
 return outcome;
   }
 
   /**
-   * Currently in order to accurately predict memory usage for spilling, the 
first non-empty build or probe side batch is needed. This method
-   * fetches the first non-empty batch from the probe or build side.
-   * @param curr The current outcome.
-   * @param inputIndex Index specifying whether to work with the prorbe or 
build input.
-   * @param recordBatch The probe or build record batch.
-   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} 
for the left or right record batch.
+   * Currently in order to accurately predict memory usage for spilling, the
+   * first non-empty build or probe side batch is needed. This method fetches
+   * the first non-empty batch from the probe or build side.
+   *
+   * @param curr
 
 Review comment:
   This code is full of such abbreviations. Out of scope to fix in this 
go-round.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037563#comment-17037563
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379848985
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -679,67 +715,91 @@ public IterOutcome innerNext() {
   }
 
   /**
-   * In case an upstream data is no longer needed, send a kill and flush any 
remaining batch
+   * In case an upstream data is no longer needed, send a kill and flush any
+   * remaining batch
*
-   * @param batch probe or build batch
-   * @param upstream which upstream
-   * @param isLeft is it the left or right
+   * @param batch
+   *  probe or build batch
+   * @param upstream
+   *  which upstream
+   * @param isLeft
+   *  is it the left or right
*/
-  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream, 
boolean isLeft) {
-batch.kill(true);
-while (upstream == IterOutcome.OK_NEW_SCHEMA || upstream == 
IterOutcome.OK) {
+  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream,
+  boolean isLeft) {
+batch.cancel();
+while (upstream == IterOutcome.OK_NEW_SCHEMA
+|| upstream == IterOutcome.OK) {
   VectorAccessibleUtilities.clear(batch);
-  upstream = next( isLeft ? HashJoinHelper.LEFT_INPUT : 
HashJoinHelper.RIGHT_INPUT, batch);
+  upstream = next(
+  isLeft ? HashJoinHelper.LEFT_INPUT : HashJoinHelper.RIGHT_INPUT,
+  batch);
 }
   }
 
-  private void killAndDrainLeftUpstream() { killAndDrainUpstream(probeBatch, 
leftUpstream, true); }
-  private void killAndDrainRightUpstream() { killAndDrainUpstream(buildBatch, 
rightUpstream, false); }
+  private void killAndDrainLeftUpstream() {
+killAndDrainUpstream(probeBatch, leftUpstream, true);
+  }
+
+  private void killAndDrainRightUpstream() {
+killAndDrainUpstream(buildBatch, rightUpstream, false);
+  }
 
   private void setupHashTable() {
-List comparators = 
Lists.newArrayListWithExpectedSize(conditions.size());
-
conditions.forEach(cond->comparators.add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
+List comparators = Lists
+.newArrayListWithExpectedSize(conditions.size());
+conditions.forEach(cond -> comparators
+.add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
 
 Review comment:
   As above.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-15 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037565#comment-17037565
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379849248
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -448,7 +445,7 @@ public boolean shouldSpill() {
 
   if (reserveHash) {
 // Include the hash sizes for the batch
-consumedMemory += ((long) IntVector.VALUE_WIDTH) * 
partitionStatsSet.getNumInMemoryRecords();
+consumedMemory += (IntVector.VALUE_WIDTH) * 
partitionStatsSet.getNumInMemoryRecords();
 
 Review comment:
   This resulted from my IDE's efforts to remove unneeded casts. It is not 
smart enough to remove unneeded parenthesis. Fixed.
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037058#comment-17037058
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379447867
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -679,67 +715,91 @@ public IterOutcome innerNext() {
   }
 
   /**
-   * In case an upstream data is no longer needed, send a kill and flush any 
remaining batch
+   * In case an upstream data is no longer needed, send a kill and flush any
+   * remaining batch
*
-   * @param batch probe or build batch
-   * @param upstream which upstream
-   * @param isLeft is it the left or right
+   * @param batch
+   *  probe or build batch
+   * @param upstream
+   *  which upstream
+   * @param isLeft
+   *  is it the left or right
*/
-  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream, 
boolean isLeft) {
-batch.kill(true);
-while (upstream == IterOutcome.OK_NEW_SCHEMA || upstream == 
IterOutcome.OK) {
+  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream,
+  boolean isLeft) {
+batch.cancel();
+while (upstream == IterOutcome.OK_NEW_SCHEMA
+|| upstream == IterOutcome.OK) {
   VectorAccessibleUtilities.clear(batch);
-  upstream = next( isLeft ? HashJoinHelper.LEFT_INPUT : 
HashJoinHelper.RIGHT_INPUT, batch);
+  upstream = next(
+  isLeft ? HashJoinHelper.LEFT_INPUT : HashJoinHelper.RIGHT_INPUT,
+  batch);
 }
   }
 
-  private void killAndDrainLeftUpstream() { killAndDrainUpstream(probeBatch, 
leftUpstream, true); }
-  private void killAndDrainRightUpstream() { killAndDrainUpstream(buildBatch, 
rightUpstream, false); }
+  private void killAndDrainLeftUpstream() {
+killAndDrainUpstream(probeBatch, leftUpstream, true);
+  }
+
+  private void killAndDrainRightUpstream() {
+killAndDrainUpstream(buildBatch, rightUpstream, false);
+  }
 
   private void setupHashTable() {
-List comparators = 
Lists.newArrayListWithExpectedSize(conditions.size());
-
conditions.forEach(cond->comparators.add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
+List comparators = Lists
+.newArrayListWithExpectedSize(conditions.size());
+conditions.forEach(cond -> comparators
+.add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
 
 Review comment:
   makes sense to collect comparators right before HashTableConfig(...) 
constructor call, using : 
   ```java
   List comparators = conditions.stream()
   .map(JoinUtils::checkAndReturnSupportedJoinComparator)
   .collect(Collectors.toList());
   ``` 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037062#comment-17037062
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379492712
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
 ##
 @@ -88,6 +88,7 @@ public void TestMultipleSendLocationBroadcastExchange() 
throws Exception {
 }
 b.release();
   }
+  // Nothing done with count?
 
 Review comment:
   maybe check count > 0 ? :)
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037059#comment-17037059
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379484930
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
 ##
 @@ -145,8 +145,7 @@ public boolean hasNext() {
 lastResultOutcome = operator.next();
 needToGrabNext = false;
   }
-  if (lastResultOutcome == RecordBatch.IterOutcome.NONE
-|| lastResultOutcome == RecordBatch.IterOutcome.STOP) {
+  if (lastResultOutcome == RecordBatch.IterOutcome.NONE) {
 
 Review comment:
   ```suggestion
 return lastResultOutcome != RecordBatch.IterOutcome.NONE;
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037065#comment-17037065
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379462590
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -1344,74 +1483,95 @@ private void updateStats() {
 stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
 stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
 stats.setLongStat(Metric.NUM_PARTITIONS, numPartitions);
-stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 
in case no spill
+stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in
+// case no
+// spill
 stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
   }
 
   /**
-   * Get the hash table iterator that is created for the build side of the 
hash join if
-   * this hash join was instantiated as a row-key join.
-   * @return hash table iterator or null if this hash join was not a row-key 
join or if it
-   * was a row-key join but the build has not yet completed.
+   * Get the hash table iterator that is created for the build side of the hash
+   * join if this hash join was instantiated as a row-key join.
+   *
+   * @return hash table iterator or null if this hash join was not a row-key
+   * join or if it was a row-key join but the build has not yet
+   * completed.
*/
   @Override
   public Pair nextRowKeyBatch() {
 if (buildComplete) {
-  // partition 0 because Row Key Join has only a single partition - no 
spilling
+  // partition 0 because Row Key Join has only a single partition - no
+  // spilling
   Pair pp = partitions[0].nextBatch();
   if (pp != null) {
 VectorWrapper vw = Iterables.get(pp.getLeft(), 0);
 ValueVector vv = vw.getValueVector();
 return Pair.of(vv, pp.getRight());
   }
-} else if(partitions == null && firstOutputBatch) { //if there is data 
coming to right(build) side in build Schema stage, use it.
+} else if (partitions == null && firstOutputBatch) { // if there is data
+ // coming to
+ // right(build) side 
in
+ // build Schema stage,
+ // use it.
 
 Review comment:
   maybe simply put line below ```else if``` ?
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037050#comment-17037050
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379430005
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -386,52 +398,52 @@ protected void buildSchema() {
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstBuildBatch() {
-rightUpstream = prefetchFirstBatch(rightUpstream,
-  prefetchedBuild,
-  buildSideIsEmpty,
-  RIGHT_INDEX,
-  buildBatch,
-  () -> {
-batchMemoryManager.update(RIGHT_INDEX, 0, true);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
-batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
-getRecordBatchStatsContext());
-  });
+rightUpstream = prefetchFirstBatch(rightUpstream, prefetchedBuild,
+buildSideIsEmpty, RIGHT_INDEX, buildBatch, () -> {
+  batchMemoryManager.update(RIGHT_INDEX, 0, true);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
+  batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
+  getRecordBatchStatsContext());
+});
   }
 
   /**
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstProbeBatch() {
-leftUpstream =  prefetchFirstBatch(leftUpstream,
-  prefetchedProbe,
-  probeSideIsEmpty,
-  LEFT_INDEX,
-  probeBatch,
-  () -> {
-batchMemoryManager.update(LEFT_INDEX, 0);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
-batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
-getRecordBatchStatsContext());
-  });
+leftUpstream = prefetchFirstBatch(leftUpstream, prefetchedProbe,
+probeSideIsEmpty, LEFT_INDEX, probeBatch, () -> {
+  batchMemoryManager.update(LEFT_INDEX, 0);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
+  batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
+  getRecordBatchStatsContext());
+});
   }
 
   /**
-   * Used to fetch the first data holding batch from either the build or probe 
side.
-   * @param outcome The current upstream outcome for either the build or probe 
side.
-   * @param prefetched A flag indicating if we have already done a prefetch of 
the first data holding batch for the probe or build side.
-   * @param isEmpty A flag indicating if the probe or build side is empty.
-   * @param index The upstream index of the probe or build batch.
-   * @param batch The probe or build batch itself.
-   * @param memoryManagerUpdate A lambda function to execute the memory 
manager update for the probe or build batch.
-   * @return The current {@link 
org.apache.drill.exec.record.RecordBatch.IterOutcome}.
+   * Used to fetch the first data holding batch from either the build or probe
+   * side.
+   *
+   * @param outcome
+   *  The current upstream outcome for either the build or probe side.
+   * @param prefetched
+   *  A flag indicating if we have already done a prefetch of the first
+   *  data holding batch for the probe or build side.
+   * @param isEmpty
+   *  A flag indicating if the probe or build side is empty.
+   * @param index
+   *  The upstream index of the probe or build batch.
+   * @param batch
+   *  The probe or build batch itself.
+   * @param memoryManagerUpdate
+   *  A lambda function to execute the memory manager update for the
+   *  probe or build batch.
+   * @return The current
+   * {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}.
*/
   private IterOutcome prefetchFirstBatch(IterOutcome outcome,
- MutableBoolean prefetched,
- MutableBoolean isEmpty,
- int index,
- RecordBatch batch,
- Runnable memoryManagerUpdate) {
+  MutableBoolean prefetched, MutableBoolean isEmpty, int index,
 
 Review comment:
   Such shallow indentations may confuse readers since at first glance look 
like method body started here. I would suggest 
   
   ```java
 private IterOutcome prefetchFirstBatch(IterOutcome outcome, MutableBoolean 
prefetched,
MutableBoolean isEmpty, int index, 
RecordBatch batch,
Runnable memoryManagerUpdate) {
   ```
 

This is an automated 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037061#comment-17037061
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379481280
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -448,7 +445,7 @@ public boolean shouldSpill() {
 
   if (reserveHash) {
 // Include the hash sizes for the batch
-consumedMemory += ((long) IntVector.VALUE_WIDTH) * 
partitionStatsSet.getNumInMemoryRecords();
+consumedMemory += (IntVector.VALUE_WIDTH) * 
partitionStatsSet.getNumInMemoryRecords();
 
 Review comment:
   ```suggestion
   consumedMemory += IntVector.VALUE_WIDTH * 
partitionStatsSet.getNumInMemoryRecords();
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037055#comment-17037055
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379438509
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -515,99 +536,95 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
   @Override
   public IterOutcome innerNext() {
 if (wasKilled) {
-  // We have received a kill signal. We need to stop processing.
-  this.cleanup();
-  super.close();
+  // We have received a cancel signal. We need to stop processing.
+  cleanup();
   return IterOutcome.NONE;
 }
 
 prefetchFirstBuildBatch();
 
 if (rightUpstream.isError()) {
-  // A termination condition was reached while prefetching the first build 
side data holding batch.
+  // A termination condition was reached while prefetching the first build
+  // side data holding batch.
   // We need to terminate.
   return rightUpstream;
 }
 
 try {
-  /* If we are here for the first time, execute the build phase of the
-   * hash join and setup the run time generated class for the probe side
+  /*
+   * If we are here for the first time, execute the build phase of the hash
+   * join and setup the run time generated class for the probe side
*/
   if (state == BatchState.FIRST) {
 // Build the hash table, using the build side record batches.
 IterOutcome buildExecuteTermination = executeBuildPhase();
 
 if (buildExecuteTermination != null) {
-  // A termination condition was reached while executing the build 
phase.
+  // A termination condition was reached while executing the build
+  // phase.
   // We need to terminate.
   return buildExecuteTermination;
 }
 
 buildComplete = true;
 
 if (isRowKeyJoin) {
-  // discard the first left batch which was fetched by buildSchema, 
and get the new
+  // discard the first left batch which was fetched by buildSchema, and
+  // get the new
 
 Review comment:
   merge with line below
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037063#comment-17037063
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379480608
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -361,13 +358,13 @@ private void calculateMemoryUsage()
 partitionProbeBatchSize = 
probeSizePredictor.predictBatchSize(recordsPerPartitionBatchProbe, reserveHash);
   }
 
-  maxOutputBatchSize = (long) ((double)outputBatchSize * 
fragmentationFactor * safetyFactor);
+  maxOutputBatchSize = (long) (outputBatchSize * fragmentationFactor * 
safetyFactor);
 
   long probeReservedMemory = 0;
 
   for (partitions = initialPartitions;; partitions /= 2) {
 // The total amount of memory to reserve for incomplete batches across 
all partitions
-long incompletePartitionsBatchSizes = ((long) partitions) * 
partitionBuildBatchSize;
+long incompletePartitionsBatchSizes = (partitions) * 
partitionBuildBatchSize;
 
 Review comment:
   ```suggestion
   long incompletePartitionsBatchSizes = partitions * 
partitionBuildBatchSize;
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037064#comment-17037064
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379438809
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -515,99 +536,95 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
   @Override
   public IterOutcome innerNext() {
 if (wasKilled) {
-  // We have received a kill signal. We need to stop processing.
-  this.cleanup();
-  super.close();
+  // We have received a cancel signal. We need to stop processing.
+  cleanup();
   return IterOutcome.NONE;
 }
 
 prefetchFirstBuildBatch();
 
 if (rightUpstream.isError()) {
-  // A termination condition was reached while prefetching the first build 
side data holding batch.
+  // A termination condition was reached while prefetching the first build
+  // side data holding batch.
   // We need to terminate.
   return rightUpstream;
 }
 
 try {
-  /* If we are here for the first time, execute the build phase of the
-   * hash join and setup the run time generated class for the probe side
+  /*
+   * If we are here for the first time, execute the build phase of the hash
+   * join and setup the run time generated class for the probe side
*/
   if (state == BatchState.FIRST) {
 // Build the hash table, using the build side record batches.
 IterOutcome buildExecuteTermination = executeBuildPhase();
 
 if (buildExecuteTermination != null) {
-  // A termination condition was reached while executing the build 
phase.
+  // A termination condition was reached while executing the build
+  // phase.
   // We need to terminate.
   return buildExecuteTermination;
 }
 
 buildComplete = true;
 
 if (isRowKeyJoin) {
-  // discard the first left batch which was fetched by buildSchema, 
and get the new
+  // discard the first left batch which was fetched by buildSchema, and
+  // get the new
   // one based on rowkey join
   leftUpstream = next(left);
-
-  if (leftUpstream == IterOutcome.STOP || rightUpstream == 
IterOutcome.STOP) {
-state = BatchState.STOP;
-return leftUpstream;
-  }
 }
 
 // Update the hash table related stats for the operator
 updateStats();
   }
 
   // Try to probe and project, or recursively handle a spilled partition
-  if (!buildSideIsEmpty.booleanValue() ||  // If there are build-side rows
-  joinIsLeftOrFull) {  // or if this is a left/full outer join
+  if (!buildSideIsEmpty.booleanValue() || // If there are build-side rows
+  joinIsLeftOrFull) { // or if this is a left/full outer join
 
 prefetchFirstProbeBatch();
 
-if (leftUpstream.isError() ||
-( leftUpstream == NONE && ! joinIsRightOrFull )) {
-  // A termination condition was reached while prefetching the first 
probe side data holding batch.
+if (leftUpstream.isError()
+|| (leftUpstream == NONE && !joinIsRightOrFull)) {
+  // A termination condition was reached while prefetching the first
+  // probe side data holding batch.
 
 Review comment:
   merge with line below
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037056#comment-17037056
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379458174
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -1222,63 +1329,81 @@ public HashJoinBatch(HashJoinPOP popConfig, 
FragmentContext context,
 
 for (int i = 0; i < conditions.size(); i++) {
   SchemaPath rightPath = (SchemaPath) conditions.get(i).getRight();
-  PathSegment.NameSegment nameSegment = 
(PathSegment.NameSegment)rightPath.getLastSegment();
+  PathSegment.NameSegment nameSegment = (PathSegment.NameSegment) rightPath
+  .getLastSegment();
   buildJoinColumns.add(nameSegment.getPath());
   String refName = "build_side_" + i;
-  rightExpr.add(new NamedExpression(conditions.get(i).getRight(), new 
FieldReference(refName)));
+  rightExpr.add(new NamedExpression(conditions.get(i).getRight(),
+  new FieldReference(refName)));
 }
 
 this.allocator = oContext.getAllocator();
 
-numPartitions = 
(int)context.getOptions().getOption(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR);
-if ( numPartitions == 1 ) { //
-  disableSpilling("Spilling is disabled due to configuration setting of 
num_partitions to 1");
+numPartitions = (int) context.getOptions()
+.getOption(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR);
+if (numPartitions == 1) { //
 
 Review comment:
   ```suggestion
   if (numPartitions == 1) {
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037053#comment-17037053
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379455455
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -812,37 +877,43 @@ private void delayedSetup() {
* Initialize fields (that may be reused when reading spilled partitions)
*/
   private void initializeBuild() {
-baseHashTable.updateIncoming(buildBatch, probeBatch); // in case we 
process the spilled files
+baseHashTable.updateIncoming(buildBatch, probeBatch); // in case we process
+  // the spilled files
 // Recreate the partitions every time build is initialized
-for (int part = 0; part < numPartitions; part++ ) {
-  partitions[part] = new HashPartition(context, allocator, baseHashTable, 
buildBatch, probeBatch, semiJoin,
-RECORDS_PER_BATCH, spillSet, part, spilledState.getCycle(), 
numPartitions);
+for (int part = 0; part < numPartitions; part++) {
+  partitions[part] = new HashPartition(context, allocator, baseHashTable,
+  buildBatch, probeBatch, semiJoin, RECORDS_PER_BATCH, spillSet, part,
+  spilledState.getCycle(), numPartitions);
 }
 
 spilledInners = new HashJoinSpilledPartition[numPartitions];
 
   }
 
   /**
-   * Note:
-   * This method can not be called again as part of recursive call of 
executeBuildPhase() to handle spilled build partitions.
+   * Note: This method can not be called again as part of recursive call of
+   * executeBuildPhase() to handle spilled build partitions.
*/
   private void initializeRuntimeFilter() {
 if (!enableRuntimeFilter || bloomFiltersGenerated) {
   return;
 }
-runtimeFilterReporter = new 
RuntimeFilterReporter((ExecutorFragmentContext) context);
+runtimeFilterReporter = new RuntimeFilterReporter(
+(ExecutorFragmentContext) context);
 
 Review comment:
   There are a lot of new line breaks introduced in the class in situations 
where the line is not actually long. Could you please revert the changes? I 
would suggest considering line with length up to 100 symbols as normal. Phew... 
Java is so verbal:)
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037049#comment-17037049
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379417689
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
 ##
 @@ -17,14 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl.filter;
 
-import org.apache.drill.exec.record.selection.SelectionVector2;
-
 public class EvaluationPredicate {
-  private SelectionVector2 vector;
-
-  EvaluationPredicate(String pred){
-
-  }
-
-
+  EvaluationPredicate(String pred) { }
 
 Review comment:
   It seems like the class is not used anywhere. Let's remove it :)
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037057#comment-17037057
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379445283
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -679,67 +715,91 @@ public IterOutcome innerNext() {
   }
 
   /**
-   * In case an upstream data is no longer needed, send a kill and flush any 
remaining batch
+   * In case an upstream data is no longer needed, send a kill and flush any
+   * remaining batch
*
-   * @param batch probe or build batch
-   * @param upstream which upstream
-   * @param isLeft is it the left or right
+   * @param batch
+   *  probe or build batch
+   * @param upstream
+   *  which upstream
+   * @param isLeft
+   *  is it the left or right
*/
-  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream, 
boolean isLeft) {
-batch.kill(true);
-while (upstream == IterOutcome.OK_NEW_SCHEMA || upstream == 
IterOutcome.OK) {
+  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream,
+  boolean isLeft) {
 
 Review comment:
   it would be better to accept int and pass HashJoinHelper.RIGHT_INPUT or  
HashJoinHelper.LEFT_INPUT when invoked. 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037052#comment-17037052
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379438348
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -515,99 +536,95 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
   @Override
   public IterOutcome innerNext() {
 if (wasKilled) {
-  // We have received a kill signal. We need to stop processing.
-  this.cleanup();
-  super.close();
+  // We have received a cancel signal. We need to stop processing.
+  cleanup();
   return IterOutcome.NONE;
 }
 
 prefetchFirstBuildBatch();
 
 if (rightUpstream.isError()) {
-  // A termination condition was reached while prefetching the first build 
side data holding batch.
+  // A termination condition was reached while prefetching the first build
+  // side data holding batch.
 
 Review comment:
   merge with line below
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037048#comment-17037048
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379419667
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -190,35 +195,44 @@
 
   private final JoinControl joinControl;
 
-  // An iterator over the build side hash table (only applicable for row-key 
joins)
+  // An iterator over the build side hash table (only applicable for row-key
+  // joins)
   private boolean buildComplete;
 
   // indicates if we have previously returned an output batch
   private boolean firstOutputBatch = true;
 
   private int rightHVColPosition;
   private final BufferAllocator allocator;
-  // Local fields for left/right incoming - may be replaced when reading from 
spilled
+  // Local fields for left/right incoming - may be replaced when reading from
+  // spilled
   private RecordBatch buildBatch;
   private RecordBatch probeBatch;
 
   /**
-   * Flag indicating whether or not the first data holding build batch needs 
to be fetched.
+   * Flag indicating whether or not the first data holding build batch needs to
+   * be fetched.
*/
   private final MutableBoolean prefetchedBuild = new MutableBoolean(false);
   /**
-   * Flag indicating whether or not the first data holding probe batch needs 
to be fetched.
+   * Flag indicating whether or not the first data holding probe batch needs to
+   * be fetched.
*/
   private final MutableBoolean prefetchedProbe = new MutableBoolean(false);
 
   // For handling spilling
   private final SpillSet spillSet;
   HashJoinPOP popConfig;
 
-  private final int originalPartition = -1; // the partition a secondary reads 
from
-  IntVector read_right_HV_vector; // HV vector that was read from the spilled 
batch
+  private final int originalPartition = -1; // the partition a secondary reads
+// from
+  IntVector read_right_HV_vector; // HV vector that was read from the spilled
+  // batch
   private final int maxBatchesInMemory;
-  private final List probeFields = new ArrayList<>(); // keep the same 
sequence with the bloomFilters
+  private final List probeFields = new ArrayList<>(); // keep the same
+  // sequence with
+  // the
+  // bloomFilters
 
 Review comment:
   Please convert such one-line comments with introduced line breaks to regular 
Javadoc comments. 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037045#comment-17037045
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379430195
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -445,32 +457,34 @@ private IterOutcome prefetchFirstBatch(IterOutcome 
outcome,
   outcome = sniffNonEmptyBatch(outcome, index, batch);
 }
 
-isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE 
there is no data.
+isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE there
+   // is no data.
 
-if (outcome == IterOutcome.STOP) {
-  // We reached a termination state
-  state = BatchState.STOP;
-} else {
-  // Got our first batch(es)
-  if (spilledState.isFirstCycle()) {
-// Only collect stats for the first cycle
-memoryManagerUpdate.run();
-  }
-  state = BatchState.FIRST;
+// Got our first batch(es)
+if (spilledState.isFirstCycle()) {
+  // Only collect stats for the first cycle
+  memoryManagerUpdate.run();
 }
-
+state = BatchState.FIRST;
 return outcome;
   }
 
   /**
-   * Currently in order to accurately predict memory usage for spilling, the 
first non-empty build or probe side batch is needed. This method
-   * fetches the first non-empty batch from the probe or build side.
-   * @param curr The current outcome.
-   * @param inputIndex Index specifying whether to work with the prorbe or 
build input.
-   * @param recordBatch The probe or build record batch.
-   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} 
for the left or right record batch.
+   * Currently in order to accurately predict memory usage for spilling, the
+   * first non-empty build or probe side batch is needed. This method fetches
+   * the first non-empty batch from the probe or build side.
+   *
+   * @param curr
+   *  The current outcome.
+   * @param inputIndex
+   *  Index specifying whether to work with the prorbe or build input.
+   * @param recordBatch
+   *  The probe or build record batch.
+   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}
+   * for the left or right record batch.
*/
-  private IterOutcome sniffNonEmptyBatch(IterOutcome curr, int inputIndex, 
RecordBatch recordBatch) {
+  private IterOutcome sniffNonEmptyBatch(IterOutcome curr, int inputIndex,
+  RecordBatch recordBatch) {
 
 Review comment:
   same as above
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037047#comment-17037047
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379432493
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -445,32 +457,34 @@ private IterOutcome prefetchFirstBatch(IterOutcome 
outcome,
   outcome = sniffNonEmptyBatch(outcome, index, batch);
 }
 
-isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE 
there is no data.
+isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE there
+   // is no data.
 
-if (outcome == IterOutcome.STOP) {
-  // We reached a termination state
-  state = BatchState.STOP;
-} else {
-  // Got our first batch(es)
-  if (spilledState.isFirstCycle()) {
-// Only collect stats for the first cycle
-memoryManagerUpdate.run();
-  }
-  state = BatchState.FIRST;
+// Got our first batch(es)
+if (spilledState.isFirstCycle()) {
+  // Only collect stats for the first cycle
+  memoryManagerUpdate.run();
 }
-
+state = BatchState.FIRST;
 return outcome;
   }
 
   /**
-   * Currently in order to accurately predict memory usage for spilling, the 
first non-empty build or probe side batch is needed. This method
-   * fetches the first non-empty batch from the probe or build side.
-   * @param curr The current outcome.
-   * @param inputIndex Index specifying whether to work with the prorbe or 
build input.
-   * @param recordBatch The probe or build record batch.
-   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} 
for the left or right record batch.
+   * Currently in order to accurately predict memory usage for spilling, the
+   * first non-empty build or probe side batch is needed. This method fetches
+   * the first non-empty batch from the probe or build side.
+   *
+   * @param curr
 
 Review comment:
   Is this curry? :) Please rename the parameter. 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037060#comment-17037060
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379438432
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -515,99 +536,95 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
   @Override
   public IterOutcome innerNext() {
 if (wasKilled) {
-  // We have received a kill signal. We need to stop processing.
-  this.cleanup();
-  super.close();
+  // We have received a cancel signal. We need to stop processing.
+  cleanup();
   return IterOutcome.NONE;
 }
 
 prefetchFirstBuildBatch();
 
 if (rightUpstream.isError()) {
-  // A termination condition was reached while prefetching the first build 
side data holding batch.
+  // A termination condition was reached while prefetching the first build
+  // side data holding batch.
   // We need to terminate.
   return rightUpstream;
 }
 
 try {
-  /* If we are here for the first time, execute the build phase of the
-   * hash join and setup the run time generated class for the probe side
+  /*
+   * If we are here for the first time, execute the build phase of the hash
+   * join and setup the run time generated class for the probe side
*/
   if (state == BatchState.FIRST) {
 // Build the hash table, using the build side record batches.
 IterOutcome buildExecuteTermination = executeBuildPhase();
 
 if (buildExecuteTermination != null) {
-  // A termination condition was reached while executing the build 
phase.
+  // A termination condition was reached while executing the build
+  // phase.
 
 Review comment:
   merge with line below
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037043#comment-17037043
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379412442
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
 ##
 @@ -297,62 +294,64 @@ public IterOutcome innerNext() {
   }
 }
 
-if (wasKilled) { // if kill() was called before, then finish up
+if (wasKilled) { // if cancel() was called before, then finish up
   aggregator.cleanup();
-  incoming.kill(false);
   return IterOutcome.NONE;
 }
 
 // Read and aggregate records
-// ( may need to run again if the spilled partition that was read
-//   generated new partitions that were all spilled )
+// (may need to run again if the spilled partition that was read
+//  generated new partitions that were all spilled)
 AggOutcome out;
 do {
-  //
   //  Read incoming batches and process their records
-  //
   out = aggregator.doWork();
 } while (out == AggOutcome.CALL_WORK_AGAIN);
 
 switch (out) {
-case CLEANUP_AND_RETURN:
-  container.zeroVectors();
-  aggregator.cleanup();
-  state = BatchState.DONE;
-  // fall through
-case RETURN_OUTCOME:
-  // rebuilds the schema in the case of complex writer expressions,
-  // since vectors would be added to batch run-time
-  IterOutcome outcome = aggregator.getOutcome();
-  switch (outcome) {
-case OK:
-case OK_NEW_SCHEMA:
-  if (firstBatch) {
-if (CollectionUtils.isNotEmpty(complexWriters)) {
-  container.buildSchema(SelectionVectorMode.NONE);
-  outcome = IterOutcome.OK_NEW_SCHEMA;
+  case CLEANUP_AND_RETURN:
+container.zeroVectors();
+aggregator.cleanup();
+state = BatchState.DONE;
+// fall through
+  case RETURN_OUTCOME:
+// rebuilds the schema in the case of complex writer expressions,
+// since vectors would be added to batch run-time
+IterOutcome outcome = aggregator.getOutcome();
+switch (outcome) {
+  case OK:
+  case OK_NEW_SCHEMA:
+if (firstBatch) {
+  if (CollectionUtils.isNotEmpty(complexWriters)) {
+container.buildSchema(SelectionVectorMode.NONE);
+// You'd be forgiven for thinking we should always return
+// OK_NEW_SCHEMA for the first batch. It turns out, when
+// two hash aggs are stacked, we get an error if the
+// upstream one returns OK_NEW_SCHEMA first. Not sure the
+// details, only know several tests fail.
+outcome = IterOutcome.OK_NEW_SCHEMA;
+  }
+  firstBatch = false;
 }
-firstBatch = false;
-  }
-  // fall thru
-default:
-  return outcome;
-  }
-
-case UPDATE_AGGREGATOR:
-  throw UserException.unsupportedError()
-  .message(SchemaChangeException.schemaChanged(
-  "Hash aggregate does not support schema change",
-  incomingSchema,
-  incoming.getSchema()).getMessage())
-  .build(logger);
-default:
-  throw new IllegalStateException(String.format("Unknown state %s.", out));
+break;
+  default:
+}
+return outcome;
+
+  case UPDATE_AGGREGATOR:
+throw UserException.unsupportedError()
+.message(SchemaChangeException.schemaChanged(
+"Hash aggregate does not support schema change",
+incomingSchema,
+incoming.getSchema()).getMessage())
+.build(logger);
+  default:
+throw new IllegalStateException(String.format("Unknown state %s.", 
out));
 }
   }
 
   /**
-   * Creates a new Aggregator based on the current schema. If setup fails, this
+   * Creates a new aggregator based on the current schema. If setup fails, this
 
 Review comment:
   ```suggestion
  * Creates a new {@link #aggregator} based on the current schema. If setup 
fails, this
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037054#comment-17037054
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379477848
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -1344,74 +1483,95 @@ private void updateStats() {
 stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
 stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
 stats.setLongStat(Metric.NUM_PARTITIONS, numPartitions);
-stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 
in case no spill
+stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in
+// case no
+// spill
 stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
   }
 
   /**
-   * Get the hash table iterator that is created for the build side of the 
hash join if
-   * this hash join was instantiated as a row-key join.
-   * @return hash table iterator or null if this hash join was not a row-key 
join or if it
-   * was a row-key join but the build has not yet completed.
+   * Get the hash table iterator that is created for the build side of the hash
+   * join if this hash join was instantiated as a row-key join.
+   *
+   * @return hash table iterator or null if this hash join was not a row-key
+   * join or if it was a row-key join but the build has not yet
+   * completed.
*/
   @Override
   public Pair nextRowKeyBatch() {
 if (buildComplete) {
-  // partition 0 because Row Key Join has only a single partition - no 
spilling
+  // partition 0 because Row Key Join has only a single partition - no
+  // spilling
   Pair pp = partitions[0].nextBatch();
   if (pp != null) {
 VectorWrapper vw = Iterables.get(pp.getLeft(), 0);
 ValueVector vv = vw.getValueVector();
 return Pair.of(vv, pp.getRight());
   }
-} else if(partitions == null && firstOutputBatch) { //if there is data 
coming to right(build) side in build Schema stage, use it.
+} else if (partitions == null && firstOutputBatch) { // if there is data
+ // coming to
+ // right(build) side 
in
+ // build Schema stage,
+ // use it.
   firstOutputBatch = false;
-  if ( right.getRecordCount() > 0 ) {
+  if (right.getRecordCount() > 0) {
 VectorWrapper vw = Iterables.get(right, 0);
 ValueVector vv = vw.getValueVector();
-return Pair.of(vv, right.getRecordCount()-1);
+return Pair.of(vv, right.getRecordCount() - 1);
   }
 }
 return null;
   }
 
-  @Override// implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public boolean hasRowKeyBatch() {
 return buildComplete;
   }
 
-  @Override   // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public BatchState getBatchState() {
 return state;
   }
 
-  @Override  // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public void setBatchState(BatchState newState) {
 state = newState;
   }
 
   @Override
-  public void killIncoming(boolean sendUpstream) {
+  protected void cancelIncoming() {
 wasKilled = true;
-probeBatch.kill(sendUpstream);
-buildBatch.kill(sendUpstream);
+probeBatch.cancel();
+buildBatch.cancel();
   }
 
   public void updateMetrics() {
-stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_BATCH_COUNT, 
batchMemoryManager.getNumIncomingBatches(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_BATCH_BYTES, 
batchMemoryManager.getAvgInputBatchSize(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_ROW_BYTES, 
batchMemoryManager.getAvgInputRowWidth(LEFT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_RECORD_COUNT, 
batchMemoryManager.getTotalInputRecords(LEFT_INDEX));
-
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_BATCH_COUNT, 
batchMemoryManager.getNumIncomingBatches(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_BATCH_BYTES, 
batchMemoryManager.getAvgInputBatchSize(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_ROW_BYTES, 
batchMemoryManager.getAvgInputRowWidth(RIGHT_INDEX));
-stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_RECORD_COUNT, 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037051#comment-17037051
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379437279
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -479,34 +493,41 @@ private IterOutcome sniffNonEmptyBatch(IterOutcome curr, 
int inputIndex, RecordB
   curr = next(inputIndex, recordBatch);
 
   switch (curr) {
-case OK:
-  // We got a data batch
-  break;
-case NOT_YET:
-  // We need to try again
-  break;
-case EMIT:
-  throw new UnsupportedOperationException("We do not support " + EMIT);
-default:
-  // Other cases are termination conditions
-  return curr;
+  case OK:
+// We got a data batch
+break;
+  case NOT_YET:
+// We need to try again
+break;
+  case EMIT:
+throw new UnsupportedOperationException("We do not support " + EMIT);
+  default:
+// Other cases are termination conditions
+return curr;
   }
 }
   }
 
   /**
-   * Determines the memory calculator to use. If maxNumBatches is configured 
simple batch counting is used to spill. Otherwise
-   * memory calculations are used to determine when to spill.
+   * Determines the memory calculator to use. If maxNumBatches is configured
+   * simple batch counting is used to spill. Otherwise memory calculations are
+   * used to determine when to spill.
+   *
* @return The memory calculator to use.
*/
   public HashJoinMemoryCalculator getCalculatorImpl() {
 if (maxBatchesInMemory == 0) {
-  double safetyFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
-  double fragmentationFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
-  double hashTableDoublingFactor = 
context.getOptions().getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
-  String hashTableCalculatorType = 
context.getOptions().getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
-
-  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor, hashTableDoublingFactor, hashTableCalculatorType, 
semiJoin);
+  double safetyFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
+  double fragmentationFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
+  double hashTableDoublingFactor = context.getOptions()
+  .getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
+  String hashTableCalculatorType = context.getOptions()
+  .getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
+
+  return new HashJoinMemoryCalculatorImpl(safetyFactor, 
fragmentationFactor,
+  hashTableDoublingFactor, hashTableCalculatorType, semiJoin);
 
 Review comment:
   In order to shorten the method could be rewritten to : 
   
   ```java
 public HashJoinMemoryCalculator getCalculatorImpl() {
   if (maxBatchesInMemory != 0) {
 return new HashJoinMechanicalMemoryCalculator(maxBatchesInMemory);
   }
   OptionManager opts = context.getOptions();
   double safety = opts.getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
   double fragmentation = 
opts.getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
   double hashTableDoubling = 
opts.getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
   String hashTableCalcType = 
opts.getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
   return new HashJoinMemoryCalculatorImpl(safety, fragmentation, 
hashTableDoubling, hashTableCalcType, semiJoin);
 }
   ``` 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a 

[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037044#comment-17037044
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379420842
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -308,35 +324,28 @@ public boolean hasPartitionLimit() {
*/
   private final SpilledState spilledState = new 
SpilledState<>();
   private final HashJoinUpdater spilledStateUpdater = new HashJoinUpdater();
-  private HashJoinSpilledPartition spilledInners[]; // for the outer to find 
the partition
+  private HashJoinSpilledPartition spilledInners[]; // for the outer to find 
the
+// partition
 
   public enum Metric implements MetricDef {
-NUM_BUCKETS,
-NUM_ENTRIES,
-NUM_RESIZING,
-RESIZING_TIME_MS,
-NUM_PARTITIONS,
-SPILLED_PARTITIONS, // number of original partitions spilled to disk
-SPILL_MB, // Number of MB of data spilled to disk. This amount is 
first written,
-  // then later re-read. So, disk I/O is twice this amount.
-SPILL_CYCLE,   // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
-LEFT_INPUT_BATCH_COUNT,
-LEFT_AVG_INPUT_BATCH_BYTES,
-LEFT_AVG_INPUT_ROW_BYTES,
-LEFT_INPUT_RECORD_COUNT,
-RIGHT_INPUT_BATCH_COUNT,
-RIGHT_AVG_INPUT_BATCH_BYTES,
-RIGHT_AVG_INPUT_ROW_BYTES,
-RIGHT_INPUT_RECORD_COUNT,
-OUTPUT_BATCH_COUNT,
-AVG_OUTPUT_BATCH_BYTES,
-AVG_OUTPUT_ROW_BYTES,
-OUTPUT_RECORD_COUNT;
+NUM_BUCKETS, NUM_ENTRIES, NUM_RESIZING, RESIZING_TIME_MS, NUM_PARTITIONS,
+// number of original partitions spilled to disk
+SPILLED_PARTITIONS,
+SPILL_MB, // Number of MB of data spilled to disk. This amount is first
+  // written,
+  // then later re-read. So, disk I/O is twice this amount.
+SPILL_CYCLE, // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+LEFT_INPUT_BATCH_COUNT, LEFT_AVG_INPUT_BATCH_BYTES, 
LEFT_AVG_INPUT_ROW_BYTES,
+LEFT_INPUT_RECORD_COUNT, RIGHT_INPUT_BATCH_COUNT, 
RIGHT_AVG_INPUT_BATCH_BYTES,
+RIGHT_AVG_INPUT_ROW_BYTES, RIGHT_INPUT_RECORD_COUNT, OUTPUT_BATCH_COUNT,
+AVG_OUTPUT_BATCH_BYTES, AVG_OUTPUT_ROW_BYTES, OUTPUT_RECORD_COUNT;
 
 Review comment:
   Please revert the change. Although this version is more compact, it is less 
readable now. 
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-14 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17037046#comment-17037046
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

ihuzenko commented on pull request #1981: DRILL-7583: Remove STOP status from 
operator outcome
URL: https://github.com/apache/drill/pull/1981#discussion_r379428382
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -386,52 +398,52 @@ protected void buildSchema() {
* Prefetches the first build side data holding batch.
*/
   private void prefetchFirstBuildBatch() {
-rightUpstream = prefetchFirstBatch(rightUpstream,
-  prefetchedBuild,
-  buildSideIsEmpty,
-  RIGHT_INDEX,
-  buildBatch,
-  () -> {
-batchMemoryManager.update(RIGHT_INDEX, 0, true);
-RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
-batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
-getRecordBatchStatsContext());
-  });
+rightUpstream = prefetchFirstBatch(rightUpstream, prefetchedBuild,
+buildSideIsEmpty, RIGHT_INDEX, buildBatch, () -> {
+  batchMemoryManager.update(RIGHT_INDEX, 0, true);
+  RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
+  batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
+  getRecordBatchStatsContext());
+});
 
 Review comment:
   It would be nice to extract the creation of the last lambda to a new method 
and have here and in method _prefetchFirstProbeBatch()_ something like: 
   ```java
   leftUpstream = prefetchFirstBatch(leftUpstream, prefetchedProbe,
   probeSideIsEmpty, LEFT_INDEX, probeBatch, probeSideMemoryUpdater());
   ```
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (DRILL-7583) Remove STOP status in favor of fail-fast

2020-02-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/DRILL-7583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17036417#comment-17036417
 ] 

ASF GitHub Bot commented on DRILL-7583:
---

paul-rogers commented on pull request #1981: DRILL-7583: Remove STOP status 
from operator outcome
URL: https://github.com/apache/drill/pull/1981
 
 
   Removes the `IterOutcome.STOP` status and revises the `kill` methods.
   
   ## Description
   
   Operators must handle both the "happy path" and failures. The "happy path" 
is defined by the `RecordBatch.IterOutcome` enum: each operator's `next()` 
method returns this outcome to describe what happened: batch, batch with new 
schema, or EOF.
   
   Operators originally handled the error path the same way: via an 
`IterOutcome` of `STOP`. Error handling was somewhat complex:
   
   * Set a failed flag in the operator,
   * Tell upstream operators they have failed,
   * Consuming data from the upstream until it returns `NONE`,
   * Originally, sometimes calling `close()`,
   * Returning a STOP status.
   
   Each operator implemented some variation or subset of the above. Early on, 
the error protocol was source of errors. I actually wrote a detailed analysis 
of the issues several years ago, and have been chipping away at fixing the 
problems ever since.
   
   Today, the error path mostly works; is simply a source of highly complex 
code. And, as it turns out, entirely unnecessary.
   
   Drill has long supported a "fail-fast" error path based on throwing an 
exception; relying on the fragment executor to clean up the operator stack. The 
"fail-fast" system is simple (throw an exception) and has a uniform way of 
cleaning up (call `close()` once on each operator.) Recent revisions have 
converted most operators to use the simpler fail-fast strategy based on 
throwing an exception instead of using the older STOP approach.
   
   The careful reader of those PRs will have noted that, as a result, no code 
returns the `STOP` status any longer.  This PR goes to the next logical step 
and removes the old, complex, and now-unused `STOP` based path.
   
   There is a related mechanism in operators: the `kill()` method, and its 
implementation, `killIncoming()`. The original purpose appears to be that step 
above: when an operator fails, fail all its children. Things got messy when an 
operator received a `STOP` status: should it call `kill()` on its child? On 
itself? Lots of fun bugs to fix back in the day.
   
   With `STOP` retired, the main purpose of the `kill()` method disappears. 
There is, however, a second use that we retain: cancelling upstream operators 
in the "normal" case. Think of the LIMIT operator: once a `LIMIT 10` has 10 
rows, it need not ask for more. A Parquet reader, say, may still be busily 
reading columns in worker threads. The `LIMIT` wants to tell its upstream 
operators, "hey, I have all the rows I need, thanks. You can go ahead and stop 
reading more data." That is, we need a "normal case" cancellation.
   
   To make clear that `kill()` now does cancellation, this PR renames the 
method to `cancel()`.
   
   ## Documentation
   
   This is not a user-visible change, it is purely internal to the execution 
engine. (The one user benefit is that error messages might be a bit more 
precise because of earlier work.)
   
   ## Testing
   
   Reran all unit tests which revealed a few complexities that remain, such as 
the way operators handle the `cancel()` call. All tests pass. The next step is 
to run the full test suite as part of pre-commit.
   
 

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


> Remove STOP status in favor of fail-fast
> 
>
> Key: DRILL-7583
> URL: https://issues.apache.org/jira/browse/DRILL-7583
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.17.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.18.0
>
>
> The original error solution was a complex process of a) setting a failed 
> flag, b) telling all upstream operators they have failed, c) returning a 
> {{STOP}} status.  Drill has long supported a "fail-fast" error path based on 
> throwing an exception; relying on the fragment executor to clean up the 
> operator stack. Recent revisions have converted most operators to use the 
> simpler fail-fast strategy based on throwing an exception instead of using 
> the older {{STOP}} approach. This change simply removes the old, now-unused 
> {{STOP}} based path.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)