[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-27 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

Verified that TPC-DS query 72 completes execution and returns results in 4 
minutes and 6 seconds on a 4 node cluster, against parquet views SF1 data. 
Apache Drill 1.15.0 commit 8ddc9d7

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
>  Labels: ready-to-commit
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-13 Thread ASF GitHub Bot (JIRA)


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

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

asfgit closed pull request #1408: DRILL-6453: Resolve deadlock when reading 
from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
index eaccd335527..fbdc4f3b8a1 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashPartition.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.common;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.drill.common.exceptions.RetryAfterSpillException;
 import org.apache.drill.common.exceptions.UserException;
@@ -122,6 +123,7 @@
   private List inMemoryBatchStats = 
Lists.newArrayList();
   private long partitionInMemorySize;
   private long numInMemoryRecords;
+  private boolean updatedRecordsPerBatch = false;
 
   public HashPartition(FragmentContext context, BufferAllocator allocator, 
ChainedHashTable baseHashTable,
RecordBatch buildBatch, RecordBatch probeBatch,
@@ -155,6 +157,18 @@ public HashPartition(FragmentContext context, 
BufferAllocator allocator, Chained
 }
   }
 
+  /**
+   * Configure a different temporary batch size when spilling probe batches.
+   * @param newRecordsPerBatch The new temporary batch size to use.
+   */
+  public void updateProbeRecordsPerBatch(int newRecordsPerBatch) {
+Preconditions.checkArgument(newRecordsPerBatch > 0);
+Preconditions.checkState(!updatedRecordsPerBatch); // Only allow updating 
once
+Preconditions.checkState(processingOuter); // We can only update the 
records per batch when probing.
+
+recordsPerBatch = newRecordsPerBatch;
+  }
+
   /**
* Allocate a new vector container for either right or left record batch
* Add an additional special vector for the hash values
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/BatchSizePredictor.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/BatchSizePredictor.java
new file mode 100644
index 000..912e4feaf3c
--- /dev/null
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/BatchSizePredictor.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+/**
+ * This class predicts the sizes of batches given an input batch.
+ *
+ * Invariants
+ * 
+ *   The {@link BatchSizePredictor} assumes that a {@link RecordBatch} is 
in a state where it can return a valid record count.
+ * 
+ */
+public interface BatchSizePredictor {
+  /**
+   * Gets the batchSize computed in the call to {@link #updateStats()}. 
Returns 0 if {@link #hadDataLastTime()} is false.
+   * @return Gets the batchSize computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hadDataLastTime()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  long getBatchSize();
+
+  /**
+   * Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hadDataLastTime()} is false.
+   * @return Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hadDataLastTime()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  int getNumRecords();
+
+  /**
+   * True if the input batch had 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-09 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on issue #1408: DRILL-6453: Resolve deadlock when reading 
from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#issuecomment-411933587
 
 
   OK - tag it ready-to-commit ...


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-09 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on issue #1408: DRILL-6453: Resolve deadlock when reading 
from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#issuecomment-411890416
 
 
   @Ben-Zvi Thanks for the review and suggestions. The code is much cleaner now.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-09 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r209072214
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -365,17 +299,20 @@ protected void initialize(boolean autoTune,
   double loadFactor) {
   Preconditions.checkState(!firstInitialized);
   Preconditions.checkArgument(initialPartitions >= 1);
+  // If we had probe data before there should still be probe data now.
+  // If we didn't have probe data before we could get some new data now.
+  Preconditions.checkState(probeSizePredictor.hasData() && !probeEmpty || 
!probeSizePredictor.hasData());
 
 Review comment:
   Done


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-09 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r209066972
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/BatchSizePredictor.java
 ##
 @@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+/**
+ * This class predicts the sizes of batches given an input batch.
+ *
+ * Invariants
+ * 
+ *   The {@link BatchSizePredictor} assumes that a {@link RecordBatch} is 
in a state where it can return a valid record count.
+ * 
+ */
+public interface BatchSizePredictor {
+  /**
+   * Gets the batchSize computed in the call to {@link #updateStats()}. 
Returns 0 if {@link #hasData()} is false.
+   * @return Gets the batchSize computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  long getBatchSize();
+
+  /**
+   * Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @return Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  int getNumRecords();
+
+  /**
+   * True if the input batch had records in the last call to {@link 
#updateStats()}. False otherwise.
+   * @return True if the input batch had records in the last call to {@link 
#updateStats()}. False otherwise.
+   */
+  boolean hasData();
 
 Review comment:
   Renamed to hadDataLastTime


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-09 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r209066765
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -420,31 +357,32 @@ public long getMaxReservedMemory() {
 private void calculateMemoryUsage()
 {
   // Adjust based on number of records
-  maxBuildBatchSize = computeMaxBatchSizeNoHash(buildBatchSize, 
buildNumRecords,
-maxBatchNumRecordsBuild, fragmentationFactor, safetyFactor);
-  maxProbeBatchSize = computeMaxBatchSizeNoHash(probeBatchSize, 
probeNumRecords,
-maxBatchNumRecordsProbe, fragmentationFactor, safetyFactor);
-
-  // Safety factor can be multiplied at the end since these batches are 
coming from exchange operators, so no excess value vector doubling
-  partitionBuildBatchSize = computeMaxBatchSize(buildBatchSize,
-buildNumRecords,
-recordsPerPartitionBatchBuild,
-fragmentationFactor,
-safetyFactor,
-reserveHash);
+  maxBuildBatchSize = 
buildSizePredictor.predictBatchSize(maxBatchNumRecordsBuild, false);
 
-  // Safety factor can be multiplied at the end since these batches are 
coming from exchange operators, so no excess value vector doubling
-  partitionProbeBatchSize = computeMaxBatchSize(
-probeBatchSize,
-probeNumRecords,
-recordsPerPartitionBatchProbe,
-fragmentationFactor,
-safetyFactor,
-reserveHash);
+  if (probeSizePredictor.hasData()) {
+// We have probe data and we can compute the max incoming size.
+maxProbeBatchSize = 
probeSizePredictor.predictBatchSize(maxBatchNumRecordsProbe, false);
+  } else {
+// We don't have probe data
+if (probeEmpty) {
+  // We know the probe has no data, so we don't need to reserve any 
space for the incoming probe
+  maxProbeBatchSize = 0;
+} else {
+  // The probe side may have data, so assume it is the max incoming 
batch size. This assumption
+  // can fail in some cases since the batch sizing project is 
incomplete.
+  maxProbeBatchSize = maxIncomingBatchSize;
+}
+  }
+
+  partitionBuildBatchSize = 
buildSizePredictor.predictBatchSize(recordsPerPartitionBatchBuild, reserveHash);
+
+  if (probeSizePredictor.hasData()) {
+partitionProbeBatchSize = 
probeSizePredictor.predictBatchSize(recordsPerPartitionBatchProbe, reserveHash);
+  }
 
   maxOutputBatchSize = (long) ((double)outputBatchSize * 
fragmentationFactor * safetyFactor);
 
-  long probeReservedMemory;
+  long probeReservedMemory = -1;
 
 Review comment:
   Removed -1


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-08 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208706673
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -420,31 +357,32 @@ public long getMaxReservedMemory() {
 private void calculateMemoryUsage()
 {
   // Adjust based on number of records
-  maxBuildBatchSize = computeMaxBatchSizeNoHash(buildBatchSize, 
buildNumRecords,
-maxBatchNumRecordsBuild, fragmentationFactor, safetyFactor);
-  maxProbeBatchSize = computeMaxBatchSizeNoHash(probeBatchSize, 
probeNumRecords,
-maxBatchNumRecordsProbe, fragmentationFactor, safetyFactor);
-
-  // Safety factor can be multiplied at the end since these batches are 
coming from exchange operators, so no excess value vector doubling
-  partitionBuildBatchSize = computeMaxBatchSize(buildBatchSize,
-buildNumRecords,
-recordsPerPartitionBatchBuild,
-fragmentationFactor,
-safetyFactor,
-reserveHash);
+  maxBuildBatchSize = 
buildSizePredictor.predictBatchSize(maxBatchNumRecordsBuild, false);
 
-  // Safety factor can be multiplied at the end since these batches are 
coming from exchange operators, so no excess value vector doubling
-  partitionProbeBatchSize = computeMaxBatchSize(
-probeBatchSize,
-probeNumRecords,
-recordsPerPartitionBatchProbe,
-fragmentationFactor,
-safetyFactor,
-reserveHash);
+  if (probeSizePredictor.hasData()) {
+// We have probe data and we can compute the max incoming size.
+maxProbeBatchSize = 
probeSizePredictor.predictBatchSize(maxBatchNumRecordsProbe, false);
+  } else {
+// We don't have probe data
+if (probeEmpty) {
+  // We know the probe has no data, so we don't need to reserve any 
space for the incoming probe
+  maxProbeBatchSize = 0;
+} else {
+  // The probe side may have data, so assume it is the max incoming 
batch size. This assumption
+  // can fail in some cases since the batch sizing project is 
incomplete.
+  maxProbeBatchSize = maxIncomingBatchSize;
+}
+  }
+
+  partitionBuildBatchSize = 
buildSizePredictor.predictBatchSize(recordsPerPartitionBatchBuild, reserveHash);
+
+  if (probeSizePredictor.hasData()) {
+partitionProbeBatchSize = 
probeSizePredictor.predictBatchSize(recordsPerPartitionBatchProbe, reserveHash);
+  }
 
   maxOutputBatchSize = (long) ((double)outputBatchSize * 
fragmentationFactor * safetyFactor);
 
-  long probeReservedMemory;
+  long probeReservedMemory = -1;
 
 Review comment:
   Why "-1" ? Looks like the default (zero) would work as well


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-08 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208753091
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/BatchSizePredictor.java
 ##
 @@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+import org.apache.drill.exec.record.RecordBatch;
+
+/**
+ * This class predicts the sizes of batches given an input batch.
+ *
+ * Invariants
+ * 
+ *   The {@link BatchSizePredictor} assumes that a {@link RecordBatch} is 
in a state where it can return a valid record count.
+ * 
+ */
+public interface BatchSizePredictor {
+  /**
+   * Gets the batchSize computed in the call to {@link #updateStats()}. 
Returns 0 if {@link #hasData()} is false.
+   * @return Gets the batchSize computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  long getBatchSize();
+
+  /**
+   * Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @return Gets the number of records computed in the call to {@link 
#updateStats()}. Returns 0 if {@link #hasData()} is false.
+   * @throws IllegalStateException if {@link #updateStats()} was never called.
+   */
+  int getNumRecords();
+
+  /**
+   * True if the input batch had records in the last call to {@link 
#updateStats()}. False otherwise.
+   * @return True if the input batch had records in the last call to {@link 
#updateStats()}. False otherwise.
+   */
+  boolean hasData();
 
 Review comment:
   The name "hasData()" is slightly confusing, as it implies preset time. Is 
there a better name ?  "hadDataLastTime" ?


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-08 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208775875
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
 ##
 @@ -365,17 +299,20 @@ protected void initialize(boolean autoTune,
   double loadFactor) {
   Preconditions.checkState(!firstInitialized);
   Preconditions.checkArgument(initialPartitions >= 1);
+  // If we had probe data before there should still be probe data now.
+  // If we didn't have probe data before we could get some new data now.
+  Preconditions.checkState(probeSizePredictor.hasData() && !probeEmpty || 
!probeSizePredictor.hasData());
 
 Review comment:
   A simpler rewrite of the condition:  **! (probeEmpty && ! 
probeSizePredictor.hasData())** 


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-07 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on issue #1408: DRILL-6453: Resolve deadlock when reading 
from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#issuecomment-411271484
 
 
   @Ben-Zvi addressed review comments. Please take another look.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-07 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208445485
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,95 +257,134 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
 
 Review comment:
   Thanks for catching this. I have removed the sniff schema methods and used 
prefetchFirstBatchFromBothSides() instead. Please see latest commit.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-07 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208445526
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,95 +257,134 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
-  @Override
-  protected boolean prefetchFirstBatchFromBothSides() {
-if (leftUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  leftUpstream = sniffNonEmptyBatch(leftUpstream, LEFT_INDEX, left);
-}
-
-if (rightUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  rightUpstream = sniffNonEmptyBatch(rightUpstream, RIGHT_INDEX, right);
-}
-
-buildSideIsEmpty = rightUpstream == IterOutcome.NONE;
-
-if (verifyOutcomeToSetBatchState(leftUpstream, rightUpstream)) {
-  // For build side, use aggregate i.e. average row width across batches
-  batchMemoryManager.update(LEFT_INDEX, 0);
-  batchMemoryManager.update(RIGHT_INDEX, 0, true);
-
-  logger.debug("BATCH_STATS, incoming left: {}", 
batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
-  logger.debug("BATCH_STATS, incoming right: {}", 
batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
-
-  // Got our first batche(s)
-  state = BatchState.FIRST;
-  return true;
-} else {
-  return false;
-}
-  }
 
   /**
* Sniffs all data necessary to construct a schema.
* @return True if all the data necessary to construct a schema has been 
retrieved. False otherwise.
*/
   private boolean sniffNewSchemas() {
+leftUpstream = sniffNewSchema(LEFT_INDEX,
+  left,
+  () -> probeSchema = left.getSchema());
+
+rightUpstream = sniffNewSchema(RIGHT_INDEX,
+  right,
+  () -> {
+// We need to have the schema of the build side even when the build 
side is empty
+buildSchema = right.getSchema();
+// position of the new "column" for keeping the hash values (after the 
real columns)
+rightHVColPosition = right.getContainer().getNumberOfColumns();
+  });
+
+// Left and right sides must return a valid response and both sides cannot 
be NONE.
+return (!leftUpstream.isError() && !rightUpstream.isError()) &&
+  (leftUpstream != IterOutcome.NONE && rightUpstream != IterOutcome.NONE);
+  }
+
+  private IterOutcome sniffNewSchema(final int index,
+ final RecordBatch batch,
+ final Runnable schemaSetter) {
 
 Review comment:
   I have removed this method.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-07 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208401852
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,95 +257,134 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
-  @Override
-  protected boolean prefetchFirstBatchFromBothSides() {
-if (leftUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  leftUpstream = sniffNonEmptyBatch(leftUpstream, LEFT_INDEX, left);
-}
-
-if (rightUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  rightUpstream = sniffNonEmptyBatch(rightUpstream, RIGHT_INDEX, right);
-}
-
-buildSideIsEmpty = rightUpstream == IterOutcome.NONE;
-
-if (verifyOutcomeToSetBatchState(leftUpstream, rightUpstream)) {
-  // For build side, use aggregate i.e. average row width across batches
-  batchMemoryManager.update(LEFT_INDEX, 0);
-  batchMemoryManager.update(RIGHT_INDEX, 0, true);
-
-  logger.debug("BATCH_STATS, incoming left: {}", 
batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
-  logger.debug("BATCH_STATS, incoming right: {}", 
batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
-
-  // Got our first batche(s)
-  state = BatchState.FIRST;
-  return true;
-} else {
-  return false;
-}
-  }
 
   /**
* Sniffs all data necessary to construct a schema.
* @return True if all the data necessary to construct a schema has been 
retrieved. False otherwise.
*/
   private boolean sniffNewSchemas() {
+leftUpstream = sniffNewSchema(LEFT_INDEX,
+  left,
+  () -> probeSchema = left.getSchema());
+
+rightUpstream = sniffNewSchema(RIGHT_INDEX,
+  right,
+  () -> {
+// We need to have the schema of the build side even when the build 
side is empty
+buildSchema = right.getSchema();
+// position of the new "column" for keeping the hash values (after the 
real columns)
+rightHVColPosition = right.getContainer().getNumberOfColumns();
+  });
+
+// Left and right sides must return a valid response and both sides cannot 
be NONE.
+return (!leftUpstream.isError() && !rightUpstream.isError()) &&
+  (leftUpstream != IterOutcome.NONE && rightUpstream != IterOutcome.NONE);
+  }
+
+  private IterOutcome sniffNewSchema(final int index,
+ final RecordBatch batch,
+ final Runnable schemaSetter) {
 
 Review comment:
   The use of lambda function as a parameter just makes this code harder to 
read, and adds no real value. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-07 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r208401247
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,95 +257,134 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
 
 Review comment:
   It would be nicer if *buildSchema()* was calling the "standard" 
*prefetchFirstBatchFromBothSides()* like the other binary operators (sans MJ). 
Instead of having specialized different code, and modifying the RecordBatch 
just for the hash join. 



This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-02 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r207435469
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -381,16 +409,14 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
 
   @Override
   public IterOutcome innerNext() {
-if (!prefetched) {
+if (!prefetchedBuild) {
   // If we didn't retrieve our first data hold batch, we need to do it now.
-  prefetched = true;
-  prefetchFirstBatchFromBothSides();
+  prefetchedBuild = true;
+  prefetchFirstBuildBatch();
 
   // Handle emitting the correct outcome for termination conditions
-  // Use the state set by prefetchFirstBatchFromBothSides to emit the 
correct termination outcome.
+  // Use the state set by prefetchFirstBuildBatch to emit the correct 
termination outcome.
 
 Review comment:
   Refactored this code.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-02 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on issue #1408: DRILL-6453: Resolve deadlock when reading 
from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#issuecomment-410138161
 
 
   @Ben-Zvi I've refactored the code, should be much cleaner now. Please take 
another look.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-08-02 Thread ASF GitHub Bot (JIRA)


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

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

ilooner commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r207435292
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,32 +254,54 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
-  @Override
-  protected boolean prefetchFirstBatchFromBothSides() {
-if (leftUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  leftUpstream = sniffNonEmptyBatch(leftUpstream, LEFT_INDEX, left);
-}
-
+  private void prefetchFirstBuildBatch() {
 if (rightUpstream != IterOutcome.NONE) {
   // We can only get data if there is data available
   rightUpstream = sniffNonEmptyBatch(rightUpstream, RIGHT_INDEX, right);
 }
 
 buildSideIsEmpty = rightUpstream == IterOutcome.NONE;
 
-if (verifyOutcomeToSetBatchState(leftUpstream, rightUpstream)) {
+if (rightUpstream == IterOutcome.OUT_OF_MEMORY) {
+  // We reached a termination state
+  state = BatchState.OUT_OF_MEMORY;
+} else if (rightUpstream == IterOutcome.STOP) {
+  state = BatchState.STOP;
+} else {
   // For build side, use aggregate i.e. average row width across batches
-  batchMemoryManager.update(LEFT_INDEX, 0);
   batchMemoryManager.update(RIGHT_INDEX, 0, true);
-
-  logger.debug("BATCH_STATS, incoming left: {}", 
batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
   logger.debug("BATCH_STATS, incoming right: {}", 
batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
 
   // Got our first batche(s)
   state = BatchState.FIRST;
+}
+  }
+
+  /**
+   *
+   * @return True terminate. False continue.
+   */
+  private boolean prefetchFirstProbeBatch() {
 
 Review comment:
   Refactored this code.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-31 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r206742825
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -248,32 +254,54 @@ protected void buildSchema() throws 
SchemaChangeException {
 }
   }
 
-  @Override
-  protected boolean prefetchFirstBatchFromBothSides() {
-if (leftUpstream != IterOutcome.NONE) {
-  // We can only get data if there is data available
-  leftUpstream = sniffNonEmptyBatch(leftUpstream, LEFT_INDEX, left);
-}
-
+  private void prefetchFirstBuildBatch() {
 if (rightUpstream != IterOutcome.NONE) {
   // We can only get data if there is data available
   rightUpstream = sniffNonEmptyBatch(rightUpstream, RIGHT_INDEX, right);
 }
 
 buildSideIsEmpty = rightUpstream == IterOutcome.NONE;
 
-if (verifyOutcomeToSetBatchState(leftUpstream, rightUpstream)) {
+if (rightUpstream == IterOutcome.OUT_OF_MEMORY) {
+  // We reached a termination state
+  state = BatchState.OUT_OF_MEMORY;
+} else if (rightUpstream == IterOutcome.STOP) {
+  state = BatchState.STOP;
+} else {
   // For build side, use aggregate i.e. average row width across batches
-  batchMemoryManager.update(LEFT_INDEX, 0);
   batchMemoryManager.update(RIGHT_INDEX, 0, true);
-
-  logger.debug("BATCH_STATS, incoming left: {}", 
batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
   logger.debug("BATCH_STATS, incoming right: {}", 
batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
 
   // Got our first batche(s)
   state = BatchState.FIRST;
+}
+  }
+
+  /**
+   *
+   * @return True terminate. False continue.
+   */
+  private boolean prefetchFirstProbeBatch() {
 
 Review comment:
   To reduce code duplication: Can have a generic "prefetch first" method, 
combining this method with *prefetchFirstBuildBatch()* and returning the up 
stream. (and set the "empty" boolean after that.)


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-31 Thread ASF GitHub Bot (JIRA)


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

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

Ben-Zvi commented on a change in pull request #1408: DRILL-6453: Resolve 
deadlock when reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408#discussion_r206743497
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
 ##
 @@ -381,16 +409,14 @@ public HashJoinMemoryCalculator getCalculatorImpl() {
 
   @Override
   public IterOutcome innerNext() {
-if (!prefetched) {
+if (!prefetchedBuild) {
   // If we didn't retrieve our first data hold batch, we need to do it now.
-  prefetched = true;
-  prefetchFirstBatchFromBothSides();
+  prefetchedBuild = true;
+  prefetchFirstBuildBatch();
 
   // Handle emitting the correct outcome for termination conditions
-  // Use the state set by prefetchFirstBatchFromBothSides to emit the 
correct termination outcome.
+  // Use the state set by prefetchFirstBuildBatch to emit the correct 
termination outcome.
 
 Review comment:
   Code cleaning: The check below (switch statement) can be done as part of the 
identical check after executeBuildPhase() (which is skipped anyway if STOP).  
Also maybe move the "wasKilled" check first thing in innerNext().


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-30 Thread ASF GitHub Bot (JIRA)


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

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

ilooner opened a new pull request #1408: DRILL-6453: Resolve deadlock when 
reading from build and probe sides simultaneously in HashJoin
URL: https://github.com/apache/drill/pull/1408
 
 
   # The Problem
   
   Originally hash join sniffed the first data holding batch in the probe and 
build side. Using the size statistics from both sides, memory calculations were 
performed in order to determine when to spill data. 
   
   The issue with this is that fetching the first data holding batch from both 
sides can cause a deadlock in the exchange operators. Details of how this can 
happen have been included by others on the Jira.
   
   # Theory of Operation
   
   ## Sniffing Batches
   
   Batch sniffing is done in three phases.
   
1. Schema sniffing is done in buildSchema()
2. Before executing the build phase we sniff the first data holding build 
side batch and use the stats to decide the number of partitions and do memory 
calculations.
3. Before executing the probe phase we sniff the first data holding probe 
side batch, and use the size statistics to do memory calculations that decide 
when to spill.
   
   ## Memory Estimation
   
   When sniffing the schema for the build and probe side, we may get lucky and 
get data for the probe side. If this is the case then we cause use the probe 
side data to estimate the optimal number of partitions to use in the join 
operator. If we don't have probe side data when computing the number of 
partitions to use we assume that the incoming probe batches will be less than 
or equal to the configured batch size.
   
   Since the number of partitions must be configured upfront before we may have 
probe data, we may get stuck in a situation where we have too many partitions 
to effectively process the probe side. In order to avoid this scenario we also 
adjust the number of records in probe side partition batches after we receive 
data from the probe side.
   
   ## Corner cases
   
   While implementing this many corner cases had to be handled.
   
   - Empty build side
   - Empty probe side
   - Empty probe and build sides
   - Getting probe side data when retrieving the probe schema
   - Not getting probe side data when retrieving the probe schema
   
   ## Testing
   
   I added unit tests for all the corner cases, and have extracted logic for 
predicting incoming and partition batch sizes into BatchSizePredictorImpl. In 
unit tests various corner cases are tested by providing mock implementation of 
BatchSizePredictorImpl.


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-18 Thread Pritesh Maker (JIRA)


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

Pritesh Maker commented on DRILL-6453:
--

My understanding is that this query did not work in Apache Drill 1.13 either 
(with OOM errors, because we didn't have hash join spilling), but it regressed 
during the 1.14 development. I think we need to still fix this correctly and 
not rush into a fix. we should also do additional testing. 

My recommendation is that [~timothyfarkas] should continue to debug/ fix the 
issue but we should start the release process and not consider this as a 
blocker. [~salim] , [~ben-zvi], [~timothyfarkas] - I hope you'll agree as well. 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-16 Thread Timothy Farkas (JIRA)


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

Timothy Farkas commented on DRILL-6453:
---

Notes about the side effects of the fix:

It's important to note that there is no real solution for this. Specifically 
there are two problems:

 1. In order to avoid a deadlock, we are forced to make the assumption that the 
batch size is honored for incoming probe batches. In some cases that assumption 
will be violated and probe batches will be larger, and queries will OOM. At 
best we can provide a tuning factor to allow users to tune their way out of 
such cases for now. As the batch sizing project progresses
 2. Since we need foreknowledge of probe batch sizes in order to prevent an OOM 
and we are assuming that probe batch sizes are the batch size (ex. 16 mb), 
there will be cases where incoming probe batches will actually be smaller. In 
these cases we will have spilled the build side to make room for probe batches 
unnecessarily, so we will see a performance regression. Discussing with Salim, 
this can be avoided for simple cases where the non-key columns in the probe 
batch are fixed length. But if the non-key columns for the probe are variable 
length then the user could decrease the configured batch size.

Sniffing both the probe and batch side prevented these two issues, but as Salim 
explained this leads to deadlock due to the fundamental exchange design. So we 
now have to make these two sacrifices in order to avoid a deadlock in the 
exchanges.

In summary, with a fix we will have to live the following issues for now:

 1. OOM in queries that use non-batch sized operators. And require user tuning 
to get the query to run
 2. Experience a performance regression when the probe side records are small 
and contain varchar non-key columns. The user could tune the batch size to 
reduce the performance impact.


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Timothy Farkas
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-13 Thread Vlad Rozov (JIRA)


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

Vlad Rozov commented on DRILL-6453:
---

A deadlock when an operator such as hash join switches between reading from 
left and right sides is caused by:

- Drill senders can send only one batch at a time. For senders such as 
broadcast or hash partitioner, it means that if one of the receivers did not 
acknowledge 3 batches, the sender won't be able to send to any of it's 
receivers and would block until the receiver sends an acknowledgment (for 
previously sent batches).
- On the receiving side, if for example hash join flips between reading from 
left and right sides, it may lead to a condition where for one minor fragment, 
left side is empty while for another minor fragment the right side is empty.
- Drill does not allow to probe if receiver queue is empty, so the first 
fragment would block waiting for the left side to become not empty, while the 
second minor fragment would block on the same condition for the right side.
- As hash join reads from left and right sides on the same thread, when it 
blocks reading from left side, right side may become full and no more 
acknowledgments would be sent to the sender. The same for the second minor 
fragment with left and right flipped. And Drill is at the deadlock condition as 
neither receiver or sender can proceed.
 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-13 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

Results of executing the simplified query with the first three joins starting 
from the leaf level in the plan, of TPC-DS query 72.
Total time taken for below query to complete was 07 min 46.719 sec

{noformat}
SELECT
 Count(*) total_cnt 
FROM catalog_sales 
JOIN inventory 
 ON ( cs_item_sk = inv_item_sk ) 
JOIN customer_demographics 
 ON ( cs_bill_cdemo_sk = cd_demo_sk ) 
JOIN household_demographics 
 ON ( cs_bill_hdemo_sk = hd_demo_sk ) 
WHERE inv_quantity_on_hand < cs_quantity 
 AND hd_buy_potential = '501-1000' 
 AND cd_marital_status = 'M' 
LIMIT 100
{noformat}

{noformat}
00-00 Screen : rowType = RecordType(BIGINT total_cnt): rowcount = 100.0, 
cumulative cost = \{9.7136055E7 rows, 6.08208382E8 cpu, 0.0 io, 9.4473289728E10 
network, 3.04611648E7 memory}, id = 2694
00-01 Project(total_cnt=[$0]) : rowType = RecordType(BIGINT total_cnt): 
rowcount = 100.0, cumulative cost = \{9.7136045E7 rows, 6.08208372E8 cpu, 0.0 
io, 9.4473289728E10 network, 3.04611648E7 memory}, id = 2693
00-02 SelectionVectorRemover : rowType = RecordType(BIGINT total_cnt): rowcount 
= 100.0, cumulative cost = \{9.7135945E7 rows, 6.08208272E8 cpu, 0.0 io, 
9.4473289728E10 network, 3.04611648E7 memory}, id = 2692
00-03 Limit(fetch=[100]) : rowType = RecordType(BIGINT total_cnt): rowcount = 
100.0, cumulative cost = \{9.7135845E7 rows, 6.08208172E8 cpu, 0.0 io, 
9.4473289728E10 network, 3.04611648E7 memory}, id = 2691
00-04 StreamAgg(group=[{}], total_cnt=[$SUM0($0)]) : rowType = 
RecordType(BIGINT total_cnt): rowcount = 1.0, cumulative cost = \{9.7135745E7 
rows, 6.08207772E8 cpu, 0.0 io, 9.4473289728E10 network, 3.04611648E7 memory}, 
id = 2690
00-05 StreamAgg(group=[{}], total_cnt=[COUNT()]) : rowType = RecordType(BIGINT 
total_cnt): rowcount = 1.0, cumulative cost = \{9.7135744E7 rows, 6.0820776E8 
cpu, 0.0 io, 9.4473289728E10 network, 3.04611648E7 memory}, id = 2689
00-06 Project($f0=[0]) : rowType = RecordType(INTEGER $f0): rowcount = 
5872500.0, cumulative cost = \{9.1263244E7 rows, 5.3773776E8 cpu, 0.0 io, 
9.4473289728E10 network, 3.04611648E7 memory}, id = 2688
00-07 HashJoin(condition=[=($0, $1)], joinType=[inner]) : rowType = 
RecordType(ANY cs_bill_hdemo_sk, ANY hd_demo_sk): rowcount = 5872500.0, 
cumulative cost = \{8.5390744E7 rows, 5.1424776E8 cpu, 0.0 io, 9.4473289728E10 
network, 3.04611648E7 memory}, id = 2687
00-09 Project(cs_bill_hdemo_sk=[$1]) : rowType = RecordType(ANY 
cs_bill_hdemo_sk): rowcount = 5872500.0, cumulative cost = \{7.9500604E7 rows, 
4.4371944E8 cpu, 0.0 io, 9.4473289728E10 network, 3.04421568E7 memory}, id = 
2682
00-11 HashJoin(condition=[=($0, $2)], joinType=[inner]) : rowType = 
RecordType(ANY cs_bill_cdemo_sk, ANY cs_bill_hdemo_sk, ANY cd_demo_sk): 
rowcount = 5872500.0, cumulative cost = \{7.3628104E7 rows, 4.3784694E8 cpu, 
0.0 io, 9.4473289728E10 network, 3.04421568E7 memory}, id = 2681
00-14 Project(cs_bill_cdemo_sk=[$0], cs_bill_hdemo_sk=[$1]) : rowType = 
RecordType(ANY cs_bill_cdemo_sk, ANY cs_bill_hdemo_sk): rowcount = 5872500.0, 
cumulative cost = \{6.3049644E7 rows, 3.5181846E8 cpu, 0.0 io, 9.4473289728E10 
network, 2.53712448E7 memory}, id = 2676
00-17 SelectionVectorRemover : rowType = RecordType(ANY cs_bill_cdemo_sk, ANY 
cs_bill_hdemo_sk, ANY cs_item_sk, ANY cs_quantity, ANY inv_item_sk, ANY 
inv_quantity_on_hand): rowcount = 5872500.0, cumulative cost = \{5.7177144E7 
rows, 3.4007346E8 cpu, 0.0 io, 9.4473289728E10 network, 2.53712448E7 memory}, 
id = 2675
00-19 Filter(condition=[<($5, $3)]) : rowType = RecordType(ANY 
cs_bill_cdemo_sk, ANY cs_bill_hdemo_sk, ANY cs_item_sk, ANY cs_quantity, ANY 
inv_item_sk, ANY inv_quantity_on_hand): rowcount = 5872500.0, cumulative cost = 
\{5.1304644E7 rows, 3.3420096E8 cpu, 0.0 io, 9.4473289728E10 network, 
2.53712448E7 memory}, id = 2674
00-21 Project(cs_bill_cdemo_sk=[$2], cs_bill_hdemo_sk=[$3], cs_item_sk=[$4], 
cs_quantity=[$5], inv_item_sk=[$0], inv_quantity_on_hand=[$1]) : rowType = 
RecordType(ANY cs_bill_cdemo_sk, ANY cs_bill_hdemo_sk, ANY cs_item_sk, ANY 
cs_quantity, ANY inv_item_sk, ANY inv_quantity_on_hand): rowcount = 1.1745E7, 
cumulative cost = \{3.9559644E7 rows, 2.6373096E8 cpu, 0.0 io, 9.4473289728E10 
network, 2.53712448E7 memory}, id = 2673
00-22 HashJoin(condition=[=($4, $0)], joinType=[inner]) : rowType = 
RecordType(ANY inv_item_sk, ANY inv_quantity_on_hand, ANY cs_bill_cdemo_sk, ANY 
cs_bill_hdemo_sk, ANY cs_item_sk, ANY cs_quantity): rowcount = 1.1745E7, 
cumulative cost = \{2.7814644E7 rows, 1.9326096E8 cpu, 0.0 io, 9.4473289728E10 
network, 2.53712448E7 memory}, id = 2672
00-24 Scan(groupscan=[ParquetGroupScan [entries=[ReadEntryWithPath 
[path=/drill/testdata/tpcds_sf1/parquet/inventory]], 
selectionRoot=/drill/testdata/tpcds_sf1/parquet/inventory, numFiles=1, 
numRowGroups=1, 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-13 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~amansinha100] I am working on it, executing the simplified query with the 
first three joins starting from the leaf level in the plan.

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-13 Thread Aman Sinha (JIRA)


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

Aman Sinha commented on DRILL-6453:
---

[~khfaraaz]  can you also try running a simplified version of the query with 
the first say 3 joins (starting from the leaf level in the plan) ?  We should 
see what the behavior is with patterns like  hash-partitioned HJ followed by 
broadcast, broadcast HJ.

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-12 Thread Aman Sinha (JIRA)


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

Aman Sinha commented on DRILL-6453:
---

[~ben-zvi], [~dgu-atmapr]  looking at the query profile and plan, the lower 
most HashJoin is the only one that does hash partitioned join while the other 8 
joins are broadcast joins.  Since the hang happens when doing a next() on an 
UnorderedReceiver (associated with the hash exchange),  can we see if changing 
this to a broadcast join helps avoid the issue ?  The reason the hash partition 
is picked is because the right side's input (Project # 04-29) estimated row 
count is 11M which is just over the 10M broadcast threshold.  Let's see if 
bumping this up to 20M helps.  This could be a temporary workaround until we 
address the underlying issue 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-10 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

The jstacks before the failure look like normal work; the upper hash joins are 
"sniffing", and the lower ones are doing actual work (building the inner, or 
probing with the outer). None of the logs shows any spilling (though spilling 
did happen).
  We thought that DRILL-6517 was the trigger for this failure (cancel after 
2:11); but now with the fix it looks the other way - this failure was causing 
DRILL-6517 to show.


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018.txt, 
> jstack_29173_June_10_2018_b.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_c.txt, 
> jstack_29173_June_10_2018_d.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt, jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-10 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

Query 72 fails (is marked as Canceled) , it cancels it self after running for 
2hrs and 11 mins. 

jstack is attached here, from few mins just before the query canceled itself 
and after it was canceled.[^jstack_29173_June_10_2018_b.txt]

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill, 
> jstack_29173_June_10_2018.txt, jstack_29173_June_10_2018_b.txt, 
> jstack_29173_June_10_2018_c.txt, jstack_29173_June_10_2018_d.txt, 
> jstack_29173_June_10_2018_e.txt
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-06 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

Also tried disabling the MUX EXCHANGE (which was set below that hanging 
unordered receiver) – but this change did not prevent the hang (thanks 
[~dgu-atmapr]).

 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-06 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

The "main" major fragment has 9 Hash-Join ops. With the recent "early sniffing" 
code change, lower Hash-Joins have to actually start executing and probing, 
while the higher ones are just building their schema.
 Seems that the failure (DRILL-6517) happened at a fragment where HJ #2 
finished "sniffing" (i.e., by calling next() once on both sides). Then at other 
fragments (where #2 was still sniffing) the bottom Hash-Join (#9) was reading 
from unordered receiver, and hanging on that read:
{code:java}
"24c0363c-22d7-ce4b-e248-0da49342192f:frag:4:115" #156 daemon prio=10 os_prio=0 
tid=0x7f8b21d80800 nid=0x3be9 waiting on condition [0x7f8b0cff7000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x000781c2b768> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at 
java.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)
at 
java.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)
at 
org.apache.drill.exec.work.batch.UnlimitedRawBatchBuffer$UnlimitedBufferQueue.take(UnlimitedRawBatchBuffer.java:61)
at 
org.apache.drill.exec.work.batch.BaseRawBatchBuffer.getNext(BaseRawBatchBuffer.java:170)
at 
org.apache.drill.exec.physical.impl.unorderedreceiver.UnorderedReceiverBatch.getNextBatch(UnorderedReceiverBatch.java:145)
at 
org.apache.drill.exec.physical.impl.unorderedreceiver.UnorderedReceiverBatch.next(UnorderedReceiverBatch.java:163)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109)
at 
org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext(AbstractUnaryRecordBatch.java:63)
at 
org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:147)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.test.generated.HashJoinProbeGen25.executeProbePhase(HashJoinProbeTemplate.java:242)
at 
org.apache.drill.exec.test.generated.HashJoinProbeGen25.probeAndProject(HashJoinProbeTemplate.java:393)
at 
org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext(HashJoinBatch.java:348)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109)
at 
org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext(AbstractUnaryRecordBatch.java:63)
at 
org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:147)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109)
at 
org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext(AbstractUnaryRecordBatch.java:63)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:109)
at 
org.apache.drill.exec.record.AbstractUnaryRecordBatch.innerNext(AbstractUnaryRecordBatch.java:63)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 
org.apache.drill.exec.test.generated.HashJoinProbeGen25.executeProbePhase(HashJoinProbeTemplate.java:242)
at 
org.apache.drill.exec.test.generated.HashJoinProbeGen25.probeAndProject(HashJoinProbeTemplate.java:393)
at 
org.apache.drill.exec.physical.impl.join.HashJoinBatch.innerNext(HashJoinBatch.java:348)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:172)
at 
org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:119)
at 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-03 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~ben-zvi] [~priteshm]

Apache Drill 1.14.0 on a 4 node cluster, TPC-DS query 72 fails (in Canceled 
state) after running for 2hrs and 11 mins 
and we see the same Exception as before towards the end of the drillbit.log 
file.


git.commit.id.abbrev=f481a7c

{noformat}
message: "SYSTEM ERROR: IllegalStateException: Record count not set for this 
vector container\n\nFragment 4:87\n\n[Error Id: 
ed305d45-742f-48df-b1ad-6813bb5fdfc4 on qa102-48.qa.lab:31010]"
 exception {
 exception_class: "java.lang.IllegalStateException"
 message: "Record count not set for this vector container"
 stack_trace {
 class_name: "com.google.common.base.Preconditions"
 file_name: "Preconditions.java"
 line_number: 173
 method_name: "checkState"
 is_native_method: false
 }
 stack_trace {
 class_name: "org.apache.drill.exec.record.VectorContainer"
 file_name: "VectorContainer.java"
 line_number: 394
 method_name: "getRecordCount"
 is_native_method: false
 }
 stack_trace {
 class_name: "org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch"
 file_name: "RemovingRecordBatch.java"
 line_number: 49
 method_name: "getRecordCount"
 is_native_method: false
 }
 stack_trace {
 class_name: "org.apache.drill.exec.record.RecordBatchSizer"
 file_name: "RecordBatchSizer.java"
 line_number: 714
 method_name: ""
 is_native_method: false
 }
 stack_trace {
 class_name: "org.apache.drill.exec.record.RecordBatchSizer"
 file_name: "RecordBatchSizer.java"
 line_number: 686
 method_name: ""
 is_native_method: false
 }
 stack_trace {
 class_name{ "org.apache.drill.exec.record.JoinBatchMemoryManager"
 file_name: "JoinBatchMemoryManager.java"
 line_number: 74
 method_name: "update"

{noformat}

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-02 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~ben-zvi] [~priteshm] the Exception message is the same.

However, we still need to find out why the query takes so long, over 2 hours to 
execute and then fails.

I will re run the test on latest apache master to verify, if this is fixed.

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-02 Thread Pritesh Maker (JIRA)


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

Pritesh Maker commented on DRILL-6453:
--

[~khfaraaz] is this exception the same as DRILL-6517

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-07-02 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

Is there a stack trace with this Exception ?


> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-26 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~priteshm] 

Results of the other test on Apache Drill 1.14.0 commit b92f599, with the two 
options set to below values,

alter system set `planner.memory.max_query_memory_per_node` = 8589934592;
alter system set `drill.exec.hashagg.fallback.enabled` = true;

And in drill-env.sh we set direct memory to

export DRILL_MAX_DIRECT_MEMORY=$\{DRILL_MAX_DIRECT_MEMORY:-"12G"}


TPC-DS query 72 is reported to be in Canceled state on UI after having run for 
2 hr 11 min 21.958 sec

The last Exception we see in the drillbit.log is this 
 
IllegalStateException: Record count not set for this vector container

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-26 Thread Pritesh Maker (JIRA)


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

Pritesh Maker commented on DRILL-6453:
--

Thanks [~khfaraaz] - what was the result of the second test?

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-26 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~priteshm] on latest apache master commit b92f599 with DEFAULT values for 
system options and default 

DRILL_MAX_DIRECT_MEMORY we see that query is in Canceled state after running 
for 2 hr 11 mins.

As part of the other test, we will increase DRILL_MAX_DIRECT_MEMORY to 12G and 
also set below options and share test results.

alter system set `planner.memory.max_query_memory_per_node` = 10737418240;

alter system set `drill.exec.hashagg.fallback.enabled` = true;

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-25 Thread Pritesh Maker (JIRA)


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

Pritesh Maker commented on DRILL-6453:
--

[~khfaraaz] did you get a chance to test this again?

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-19 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

Thanks [~khfaraaz] for getting all the information. From the log it clearly 
shows that several Hash-Join operators in the query did not have enough memory. 
The plan must be highly parallel, and with multiple "buffered" operators; hence 
when the 10GB memory (per query per node) was divided among all of them, the 
result (memory for this instance of Hash Join) was +*very small*+ !   The 
number 40MB shown above is even bigger than that result, as we "artificially 
bump up" the result to 40MB (see the internal option 
"planner.memory.min_memory_per_buffered_op" ).

  Even with this bump up, the memory was not sufficient (needed a minimum of 
76MB). So the Hash-Join operator went into "fallback mode", that is - ignoring 
any memory constraints (just like in 1.13 and before). This fallback is also 
controlled by an option ("drill.exec.hashjoin.fallback.enabled"), which 
currently defaults to true, but we want to change the default soon to false. 
This change would cause the above query to fail with a detailed message 
(suggesting more memory is needed).

Also the Hash-Aggregate has a similar option (default - false), which the above 
shows was set to be true. We suggest customers not go this way, but rather 
allocate more memory.

So what went wrong – so many operators used "fallback" and allocated more 
memory than planned, possibly beyond 10GB in total. In case the total reached 
12GB, the JVM would have caused an OOM. This looks very much like 
h1. DRILL-6468: OOMs trigger graceful shutdown when terminating Drill. This can 
cause a hang.

whose PR ( #1306 ) was just committed into the master branch.  [~khfaraaz] – 
can you test again with this new PR included ?

 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-18 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~ben-zvi] [~priteshm] TPC-DS query 72 was executed on commit 
b447260e49dc4a8c906f5b310c037fe6dd77166f

We do not see the parquet reader error. However, we still need to investigate 
why the query takes so long (forever) to complete execution.
{noformat}
In drill-env.sh we set max direct memory to 12G on all 4nodes in cluster
export DRILL_MAX_DIRECT_MEMORY=${DRILL_MAX_DIRECT_MEMORY:-"12G"}

and also set these options from sqlline,
alter system set `planner.memory.max_query_memory_per_node` = 10737418240;
alter system set `drill.exec.hashagg.fallback.enabled` = true;{noformat}

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-15 Thread Pritesh Maker (JIRA)


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

Pritesh Maker commented on DRILL-6453:
--

[~khfaraaz] the parquet reader issue was recently fixed by [~vrozov] - could 
you please try the query 72 again?

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-06 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~ben-zvi] TPC-DS query 72 can not be executed on latest Apache master, due to 
a known issue in the Parquet reader.

[~dgu-atmapr] can you please file a Jira for the below issue ?
{noformat}
Caused by: org.apache.drill.common.exceptions.DrillRuntimeException: Error in 
parquet record reader.
Message: 
Hadoop path: 
/drill/testdata/tpcds_sf1/parquet/customer_demographics/0_0_0.parquet
Total records read: 0
Row group index: 0
Records in row group: 1920800
Parquet Metadata: ParquetMetaData{FileMetaData{schema: message root {
  optional int64 cd_demo_sk;
  optional binary cd_gender (UTF8);
  optional binary cd_marital_status (UTF8);
  optional binary cd_education_status (UTF8);
  optional int32 cd_purchase_estimate;
  optional binary cd_credit_rating (UTF8);
  optional int32 cd_dep_count;
  optional int32 cd_dep_employed_count;
  optional int32 cd_dep_college_count;
}
, metadata: {drill-writer.version=2, drill.version=1.13.0-SNAPSHOT}}, blocks: 
[BlockMetaData{1920800, 112509832 [ColumnMetaData{UNCOMPRESSED [cd_demo_sk] 
INT64  [BIT_PACKED, RLE, PLAIN], 4}, ColumnMetaData{UNCOMPRESSED [cd_gender] 
BINARY  [BIT_PACKED, RLE, PLAIN], 15367257}, ColumnMetaData{UNCOMPRESSED 
[cd_marital_status] BINARY  [BIT_PACKED, RLE, PLAIN], 24971685}, 
ColumnMetaData{UNCOMPRESSED [cd_education_status] BINARY  [BIT_PACKED, RLE, 
PLAIN], 34576113}, ColumnMetaData{UNCOMPRESSED [cd_purchase_estimate] INT32  
[BIT_PACKED, RLE, PLAIN], 60645586}, ColumnMetaData{UNCOMPRESSED 
[cd_credit_rating] BINARY  [BIT_PACKED, RLE, PLAIN], 68329176}, 
ColumnMetaData{UNCOMPRESSED [cd_dep_count] INT32  [BIT_PACKED, RLE, PLAIN], 
89459066}, ColumnMetaData{UNCOMPRESSED [cd_dep_employed_count] INT32  
[BIT_PACKED, RLE, PLAIN], 97142656}, ColumnMetaData{UNCOMPRESSED 
[cd_dep_college_count] INT32  [BIT_PACKED, RLE, PLAIN], 104826246}]}]}
at 
org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.handleException(ParquetRecordReader.java:275)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader.next(ParquetRecordReader.java:302)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.physical.impl.ScanBatch.next(ScanBatch.java:172) 
[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
... 21 common frames omitted
Caused by: java.lang.UnsupportedOperationException: Unsupoorted Operation
at 
org.apache.drill.exec.store.parquet.columnreaders.PageReader.resetDefinitionLevelReader(PageReader.java:449)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenColumnBulkInput$VLColumnBulkInputCallback.resetDefinitionLevelReader(VarLenColumnBulkInput.java:422)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenBulkPageReader.getEntry(VarLenBulkPageReader.java:113)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenColumnBulkInput.next(VarLenColumnBulkInput.java:128)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenColumnBulkInput.next(VarLenColumnBulkInput.java:32)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.vector.VarCharVector$Mutator.setSafe(VarCharVector.java:624)
 ~[vector-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.vector.NullableVarCharVector$Mutator.setSafe(NullableVarCharVector.java:719)
 ~[vector-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLengthColumnReaders$NullableVarCharColumn.setSafe(VarLengthColumnReaders.java:215)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLengthValuesColumn.readRecordsInBulk(VarLengthValuesColumn.java:98)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenBinaryReader.readRecordsInBulk(VarLenBinaryReader.java:102)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.VarLenBinaryReader.readFields(VarLenBinaryReader.java:90)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.BatchReader$VariableWidthReader.readRecords(BatchReader.java:166)
 ~[drill-java-exec-1.14.0-SNAPSHOT.jar:1.14.0-SNAPSHOT]
at 
org.apache.drill.exec.store.parquet.columnreaders.BatchReader.readBatch(BatchReader.java:42)
 

[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-06 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~ben-zvi] the query shows same behavior with or without setting (does not 
complete after 2 hours of execution)

`exec.hashjoin.num_partitions` = 1;

I will share the drillbit.log 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-06 Thread Boaz Ben-Zvi (JIRA)


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

Boaz Ben-Zvi commented on DRILL-6453:
-

[~khfaraaz] Can we have the logs for around that time (i.e. about 50 +/- 
seconds after execution started) ?   And can this query be retried without the 
"num_partition = 1" setting ? (May need to allocate more memory, to avoid OOM). 

 

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Boaz Ben-Zvi
>Priority: Blocker
> Fix For: 1.14.0
>
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-05 Thread Khurram Faraaz (JIRA)


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

Khurram Faraaz commented on DRILL-6453:
---

[~vvysotskyi] TPC-DS query 72 executed in close to 50 seconds, prior to this 
commit 

89e0fe6b34259a2f51a7c45070935a2a2400eca4

After above commit we see the increase in query execution time.

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Volodymyr Vysotskyi
>Priority: Critical
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (DRILL-6453) TPC-DS query 72 has regressed

2018-06-04 Thread Volodymyr Vysotskyi (JIRA)


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

Volodymyr Vysotskyi commented on DRILL-6453:


[~khfaraaz], if this is a regression, could you please specify a commit, which 
caused it?

> TPC-DS query 72 has regressed
> -
>
> Key: DRILL-6453
> URL: https://issues.apache.org/jira/browse/DRILL-6453
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Execution - Flow
>Affects Versions: 1.14.0
>Reporter: Khurram Faraaz
>Assignee: Volodymyr Vysotskyi
>Priority: Critical
> Attachments: 24f75b18-014a-fb58-21d2-baeab5c3352c.sys.drill
>
>
> TPC-DS query 72 seems to have regressed, query profile for the case where it 
> Canceled after 2 hours on Drill 1.14.0 is attached here.
> {noformat}
> On, Drill 1.14.0-SNAPSHOT 
> commit : 931b43e (TPC-DS query 72 executed successfully on this commit, took 
> around 55 seconds to execute)
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> TPC-DS query 72 executed successfully & took 47 seconds to complete execution.
> {noformat}
> {noformat}
> TPC-DS data in the below run has date values stored as DATE datatype and not 
> VARCHAR type
> On, Drill 1.14.0-SNAPSHOT
> commit : 82e1a12
> SF1 parquet data on 4 nodes; 
> planner.memory.max_query_memory_per_node = 10737418240. 
> drill.exec.hashagg.fallback.enabled = true
> and
> alter system set `exec.hashjoin.num_partitions` = 1;
> TPC-DS query 72 executed for 2 hrs and 11 mins and did not complete, I had to 
> Cancel it by stopping the Foreman drillbit.
> As a result several minor fragments are reported to be in 
> CANCELLATION_REQUESTED state on UI.
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)