[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99045403
  
--- Diff: exec/java-exec/src/main/resources/drill-module.conf ---
@@ -177,13 +177,47 @@ drill.exec: {
   sort: {
 purge.threshold : 1000,
 external: {
-  batch.size : 4000,
+  // Drill uses the managed External Sort Batch by default.
+  // Set this to true to use the legacy, unmanaged version.
+  // Disabled in the intial commit, to be enabled after
+  // tests are committed.
+  disable_managed: true
+  // Limit on the number of batches buffered in memory.
+  // Primarily for testing.
+  // 0 = unlimited
+  batch_limit: 0
+  // Limit on the amount of memory used for xsort. Overrides the
+  // value provided by Foreman. Primarily for testing.
+  // 0 = unlimited, Supports HOCON memory suffixes.
+  mem_limit: 0
+  // Limit on the number of spilled batches that can be merged in
+  // a single pass. Limits the number of open file handles.
+  // 0 = unlimited
+  merge_limit: 0
   spill: {
-batch.size : 4000,
-group.size : 4,
-threshold : 4,
-directories : [ "/tmp/drill/spill" ],
-fs : "file:///"
+// Deprecated for managed xsort; used only by legacy xsort
+group.size: 4,
+// Deprecated for managed xsort; used only by legacy xsort
+threshold: 4,
+// Minimum number of in-memory batches to spill per spill file
+// Affects only spilling from memory to disk.
+// Primarily for testing.
+min_batches: 2,
+// Maximum number of in-memory batches to spill per spill file
+// Affects only spilling from memory to disk.
+// Primarily for testing.
+// 0 = unlimited
+max_batches: 0,
+// File system to use. Local file system by default.
+fs: "file:///"
+// List of directories to use. Directories are created
--- End diff --

List of (comma separated) directories


> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99013960
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * 

[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99041705
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpillSet.java
 ---
@@ -0,0 +1,261 @@
+/*
+ * 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.xsort.managed;
--- End diff --

This class should be in a more general package, to be used also for other 
operators that spill (future work).



> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99027623
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
+  private final FragmentContext context;
+  private final BufferAllocator oAllocator;
+  private SelectionVector4 sv4;
+  private final OperatorCodeGenerator opCg;
+  private int batchCount;
+
+  public InMemorySorter(FragmentContext context, BufferAllocator 
allocator, OperatorCodeGenerator opCg) {
+this.context = context;
+this.oAllocator = allocator;
+this.opCg = opCg;
+  }
+
+  public SelectionVector4 sort(LinkedList 
batchGroups, VectorAccessible batch,
+VectorContainer destContainer) {
+if (builder != null) {
+  builder.clear();
+  builder.close();
+}
+builder = new SortRecordBatchBuilder(oAllocator);
+
+for (BatchGroup.InputBatch group : batchGroups) {
+  RecordBatchData rbd = new RecordBatchData(group.getContainer(), 
oAllocator);
+  rbd.setSv2(group.getSv2());
+  builder.add(rbd);
+}
+batchGroups.clear();
+
+try {
+  builder.build(context, destContainer);
+  sv4 = builder.getSv4();
+  mSorter = opCg.createNewMSorter(batch);
+  mSorter.setup(context, oAllocator, sv4, destContainer, 
sv4.getCount());
+} catch (SchemaChangeException e) {
+  throw UserException.unsupportedError(e)
+.message("Unexpected schema change - likely code error.")
+.build(logger);
+}
+
+// For testing memory-leaks, inject exception after mSorter finishes 
setup
+
ExternalSortBatch.injector.injectUnchecked(context.getExecutionControls(), 
ExternalSortBatch.INTERRUPTION_AFTER_SETUP);
+mSorter.sort(destContainer);
+
+// sort may have prematurely exited due to should continue returning 
false.
+if (!context.shouldContinue()) {
+  return null;
+}
+
+// For testing memory-leak purpose, inject exception after mSorter 
finishes sorting
+
ExternalSortBatch.injector.injectUnchecked(context.getExecutionControls(), 
ExternalSortBatch.INTERRUPTION_AFTER_SORT);
+sv4 = mSorter.getSV4();
+
+destContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
+return sv4;
+  }
+
+  @Override
+  public boolean next() {
+boolean more = sv4.next();
+if (more) { batchCount++; }
+return more;
+  }
+
+  @Override
+  public void close() {
+  

[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99024407
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * 

[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99028635
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
+  private final FragmentContext context;
+  private final BufferAllocator oAllocator;
+  private SelectionVector4 sv4;
+  private final OperatorCodeGenerator opCg;
+  private int batchCount;
+
+  public InMemorySorter(FragmentContext context, BufferAllocator 
allocator, OperatorCodeGenerator opCg) {
+this.context = context;
+this.oAllocator = allocator;
+this.opCg = opCg;
+  }
+
+  public SelectionVector4 sort(LinkedList 
batchGroups, VectorAccessible batch,
--- End diff --

maybe add some Javadoc in this file ...


> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99036541
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
 ---
@@ -0,0 +1,237 @@
+/*
+ * 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.xsort.managed;
+
+import java.util.Queue;
+
+import javax.inject.Named;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.hadoop.util.IndexedSortable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+
+import io.netty.buffer.DrillBuf;
+
+public abstract class MSortTemplate implements MSorter, IndexedSortable {
+//  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(MSortTemplate.class);
+
+  private SelectionVector4 vector4;
+  private SelectionVector4 aux;
+  @SuppressWarnings("unused")
+  private long compares;
+
+  /**
+   * Holds offsets into the SV4 of the start of each batch
+   * (sorted run.)
+   */
+
+  private Queue runStarts = Queues.newLinkedBlockingQueue();
+  private FragmentContext context;
+
+  /**
+   * Controls the maximum size of batches exposed to downstream
+   */
+  private int desiredRecordBatchCount;
+
+  @Override
+  public void setup(final FragmentContext context, final BufferAllocator 
allocator, final SelectionVector4 vector4,
+final VectorContainer hyperBatch, int outputBatchSize) 
throws SchemaChangeException{
+// we pass in the local hyperBatch since that is where we'll be 
reading data.
+Preconditions.checkNotNull(vector4);
+this.vector4 = vector4.createNewWrapperCurrent();
+this.context = context;
+vector4.clear();
+doSetup(context, hyperBatch, null);
+
+// Populate the queue with the offset in the SV4 of each
+// batch. Note that this is expensive as it requires a scan
+// of all items to be sorted: potentially millions.
+
+runStarts.add(0);
+int batch = 0;
+final int totalCount = this.vector4.getTotalCount();
+for (int i = 0; i < totalCount; i++) {
+  final int newBatch = this.vector4.get(i) >>> 16;
+  if (newBatch == batch) {
+continue;
+  } else if (newBatch == batch + 1) {
+runStarts.add(i);
+batch = newBatch;
+  } else {
+throw new UnsupportedOperationException(String.format("Missing 
batch. batch: %d newBatch: %d", batch, newBatch));
+  }
+}
+
+// Create a temporary SV4 to hold the merged results.
+
+@SuppressWarnings("resource")
+final DrillBuf drillBuf = allocator.buffer(4 * totalCount);
+desiredRecordBatchCount = Math.min(outputBatchSize, 
Character.MAX_VALUE);
+desiredRecordBatchCount = Math.min(desiredRecordBatchCount, 
totalCount);
+aux = new SelectionVector4(drillBuf, totalCount, 
desiredRecordBatchCount);
+  }
+
+  /**
+   * For given recordCount how much memory does MSorter needs for its own 
purpose. This is used in
+   * ExternalSortBatch to make decisions about whether to spill or not.
+   *
+   * @param recordCount
+   * @return
+   */
+  public static long memoryNeeded(final int recordCount) {
+// We need 4 bytes 

[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99040449
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
 ---
@@ -0,0 +1,43 @@
+/*
+ * 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.xsort.managed;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.VectorAccessible;
+
+public interface PriorityQueueCopier extends AutoCloseable {
+  public static final long INITIAL_ALLOCATION = 1000;
+  public static final long MAX_ALLOCATION = 2000;
--- End diff --

Suggestion: change to  10_000_000L and 20_000_000L 



> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99017885
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1321 @@
+/*
+ * 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.xsort.managed;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractContainerVector;
+
+import com.google.common.collect.Lists;
+
+/**
+ * External sort batch: a sort batch which can spill to disk in
+ * order to operate within a defined memory footprint.
+ * 
+ * Basic Operation
+ * The operator has three key phases:
+ * 
+ * 
+ * The load phase in which batches are read from upstream.
+ * The merge phase in which spilled batches are combined to
+ * reduce the number of files below the configured limit. (Best
+ * practice is to configure the system to avoid this phase.)
+ * The delivery phase in which batches are combined to produce
+ * the final output.
+ * 
+ * During the load phase:
+ * 
+ * 
+ * The incoming (upstream) operator provides a series of batches.
+ * This operator sorts each batch, and accumulates them in an in-memory
+ * buffer.
+ * If the in-memory buffer becomes too large, this operator selects
+ * a subset of the buffered batches to spill.
+ * Each spill set is merged to create a new, sorted collection of
+ * batches, and each is spilled to disk.
+ * To allow the use of multiple disk storage, each spill group is 
written
+ * round-robin to a set of spill directories.
+ * 
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator is complete, this operator merges the 
accumulated
+ * batches (which may be all in memory or partially on disk), and returns
+ * them to the output (downstream) operator in chunks of no more than
+ * 32K records.
+ * The final merge must combine a collection of in-memory and spilled
+ * 

[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99028564
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/InMemorySorter.java
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.xsort.managed;
+
+import java.util.LinkedList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.sort.SortRecordBatchBuilder;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+public class InMemorySorter implements SortResults {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(InMemorySorter.class);
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
--- End diff --

Can the "builder" and the "mSorter" become internal variables of sort() 
below, and then cleared/closed before sort() returns ?  And then close() would 
be a no-op.



> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user Ben-Zvi commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99039654
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
 ---
@@ -0,0 +1,271 @@
+/*
+ * 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.xsort.managed;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.exec.compile.sig.GeneratorMapping;
+import org.apache.drill.exec.compile.sig.MappingSet;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.vector.CopyUtil;
+
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+
+/**
+ * Generates and manages the data-specific classes for this operator.
+ * 
+ * Several of the code generation methods take a batch, but the methods
+ * are called for many batches, and generate code only for the first one.
+ * Better would be to generate code from a schema; but Drill is not set
+ * up for that at present.
+ */
+
+public class OperatorCodeGenerator {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(OperatorCodeGenerator.class);
+
+  protected static final MappingSet MAIN_MAPPING = new MappingSet((String) 
null, null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+  protected static final MappingSet LEFT_MAPPING = new 
MappingSet("leftIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+  protected static final MappingSet RIGHT_MAPPING = new 
MappingSet("rightIndex", null, ClassGenerator.DEFAULT_SCALAR_MAP, 
ClassGenerator.DEFAULT_SCALAR_MAP);
+
+  private static final GeneratorMapping COPIER_MAPPING = new 
GeneratorMapping("doSetup", "doCopy", null, null);
+  private static final MappingSet COPIER_MAPPING_SET = new 
MappingSet(COPIER_MAPPING, COPIER_MAPPING);
+
+  private final FragmentContext context;
+  @SuppressWarnings("unused")
+  private BatchSchema schema;
+
+  /**
+   * A single PriorityQueueCopier instance is used for 2 purposes:
+   * 1. Merge sorted batches before spilling
+   * 2. Merge sorted batches when all incoming data fits in memory
+   */
+
+  private PriorityQueueCopier copier;
+  private final Sort popConfig;
+  private MSorter mSorter;
+
+  /**
+   * Generated sort operation used to sort each incoming batch according to
+   * the sort criteria specified in the {@link ExternalSort} definition of
+   * this operator.
+   */
+
+  private SingleBatchSorter sorter;
+
+  public OperatorCodeGenerator(FragmentContext 

[jira] [Commented] (DRILL-3584) Drill Kerberos HDFS Support / Documentation

2017-02-01 Thread Pushpinder Heer (JIRA)

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

Pushpinder Heer commented on DRILL-3584:


Any updates on this ticket?

> Drill Kerberos HDFS Support / Documentation
> ---
>
> Key: DRILL-3584
> URL: https://issues.apache.org/jira/browse/DRILL-3584
> Project: Apache Drill
>  Issue Type: New Feature
>Affects Versions: 1.1.0
>Reporter: Hari Sekhon
>Priority: Critical
>  Labels: security
>
> I'm trying to find Drill docs for Kerberos support for secure HDFS clusters 
> and it doesn't appear to be well tested / supported / documented yet.
> This product is Dead-on-Arrival if it doesn't integrate well with secure 
> Hadoop clusters, specifically HDFS + Kerberos (plus obviously secure 
> kerberized Hive/HCatalog etc.)



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (DRILL-4179) Update UDF documentation now that classpath scanning is more strict

2017-02-01 Thread Julien Le Dem (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-4179?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Julien Le Dem resolved DRILL-4179.
--
Resolution: Fixed

> Update UDF documentation now that classpath scanning is more strict
> ---
>
> Key: DRILL-4179
> URL: https://issues.apache.org/jira/browse/DRILL-4179
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Documentation
>Reporter: Jason Altekruse
>Assignee: Julien Le Dem
>
> A few issues have come up with users that have UDFs that could be found with 
> 1.0-1.2, but fail to be loaded with 1.3. There were changes in 1.3 to speed 
> up finding all UDFs on the classpath made the setup a little more strict.
> Some discussions on the topic:
> DRILL-4178
> http://search-hadoop.com/m/qRVAXvthcn1xIHUm/+add+your+package+to+drill.classpath.scanning=Re+UDFs+and+1+3



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5115) Metadata Cache Pruning randomly returns wrong results at higher concurrencies

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5115?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5115:

Affects Version/s: (was: 1.10)
   1.10.0

> Metadata Cache Pruning randomly returns wrong results at higher concurrencies
> -
>
> Key: DRILL-5115
> URL: https://issues.apache.org/jira/browse/DRILL-5115
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Metadata, Query Planning & Optimization
>Affects Versions: 1.8.0, 1.9.0, 1.10.0
>Reporter: Rahul Challapalli
> Attachments: correctdata_profile.txt, drillbit1.log, drillbit2.log, 
> l_3level.tgz, wrongdata_profile.txt
>
>
> git.commit.id.abbrev=4312d65
> When multiple queries are updating the metadata cache simultaneously the 
> below query randomly returns wrong results. 
> A single run includes executing a suite of 90 tests at a concurrency of 50. I 
> encountered a wrong data scenario in my 10th run.
>  
> Query :
> {code}
> select l_orderkey from l_3level where dir0=1 and ((dir1='one' and dir2 IN 
> ('2015-7-12', '2015-7-13')) or (dir1='two' and dir2='2015-8-12'))
> {code}
> Wrong Result Plan (based on the profile) : 
> {code}
> 00-00Screen : rowType = RecordType(ANY l_orderkey): rowcount = 310.0, 
> cumulative cost = {341.0 rows, 341.0 cpu, 0.0 io, 0.0 network, 0.0 memory}, 
> id = 205721
> 00-01  Project(l_orderkey=[$0]) : rowType = RecordType(ANY l_orderkey): 
> rowcount = 310.0, cumulative cost = {310.0 rows, 310.0 cpu, 0.0 io, 0.0 
> network, 0.0 memory}, id = 205720
> 00-02Scan(groupscan=[ParquetGroupScan [entries=[ReadEntryWithPath 
> [path=maprfs:/drill/testdata/metadata_caching_pp/l_3level/1/one/2015-7-13/20.parquet],
>  ReadEntryWithPath 
> [path=maprfs:/drill/testdata/metadata_caching_pp/l_3level/1/two/2015-8-12/30.parquet],
>  ReadEntryWithPath 
> [path=maprfs:/drill/testdata/metadata_caching_pp/l_3level/1/one/2015-7-12/10.parquet]],
>  selectionRoot=maprfs:/drill/testdata/metadata_caching_pp/l_3level, 
> numFiles=3, usedMetadataFile=true, 
> cacheFileRoot=/drill/testdata/metadata_caching_pp/l_3level, 
> columns=[`l_orderkey`]]]) : rowType = RecordType(ANY l_orderkey): rowcount = 
> 310.0, cumulative cost = {310.0 rows, 310.0 cpu, 0.0 io, 0.0 network, 0.0 
> memory}, id = 205719
> {code}
> Correct Result Plan (based on the profile):
> {code}
> 00-00Screen : rowType = RecordType(ANY l_orderkey): rowcount = 2.25, 
> cumulative cost = {122.475 rows, 527.475 cpu, 0.0 io, 0.0 network, 0.0 
> memory}, id = 226849
> 00-01  Project(l_orderkey=[$3]) : rowType = RecordType(ANY l_orderkey): 
> rowcount = 2.25, cumulative cost = {122.25 rows, 527.25 cpu, 0.0 io, 0.0 
> network, 0.0 memory}, id = 226848
> 00-02SelectionVectorRemover : rowType = RecordType(ANY dir0, ANY 
> dir1, ANY dir2, ANY l_orderkey): rowcount = 2.25, cumulative cost = {122.25 
> rows, 527.25 cpu, 0.0 io, 0.0 network, 0.0 memory}, id = 226847
> 00-03  Filter(condition=[AND(=($0, 1), OR(AND(=($1, 'one'), OR(=($2, 
> '2015-7-12'), =($2, '2015-7-13'))), AND(=($1, 'two'), =($2, '2015-8-12']) 
> : rowType = RecordType(ANY dir0, ANY dir1, ANY dir2, ANY l_orderkey): 
> rowcount = 2.25, cumulative cost = {120.0 rows, 525.0 cpu, 0.0 io, 0.0 
> network, 0.0 memory}, id = 226846
> 00-04Scan(groupscan=[ParquetGroupScan [entries=[ReadEntryWithPath 
> [path=/drill/testdata/metadata_caching_pp/l_3level/1/one/2015-7-13/20.parquet],
>  ReadEntryWithPath 
> [path=/drill/testdata/metadata_caching_pp/l_3level/1/two/2015-8-12/30.parquet],
>  ReadEntryWithPath 
> [path=/drill/testdata/metadata_caching_pp/l_3level/1/one/2015-7-12/10.parquet]],
>  selectionRoot=/drill/testdata/metadata_caching_pp/l_3level, numFiles=3, 
> usedMetadataFile=true, 
> cacheFileRoot=/drill/testdata/metadata_caching_pp/l_3level/1, 
> columns=[`dir0`, `dir1`, `dir2`, `l_orderkey`]]]) : rowType = RecordType(ANY 
> dir0, ANY dir1, ANY dir2, ANY l_orderkey): rowcount = 60.0, cumulative cost = 
> {60.0 rows, 240.0 cpu, 0.0 io, 0.0 network, 0.0 memory}, id = 226845
> {code}
> I attached the data set, log files and the query profiles. Let me know if you 
> need anything



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5127) Revert the fix for DRILL-4831

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5127:

Affects Version/s: (was: 1.10)
   1.10.0

> Revert the fix for DRILL-4831
> -
>
> Key: DRILL-5127
> URL: https://issues.apache.org/jira/browse/DRILL-5127
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Metadata
>Affects Versions: 1.10.0
>Reporter: Rahul Challapalli
>Assignee: Padma Penumarthy
>  Labels: ready-to-commit
> Fix For: 1.10.0
>
>
> Git Commit # : 3f3811818ecc3bbf6f307a408c30f0406fadc703
> DRILL-4831 introduced a major regression DRILL-5082. I tested the supposed 
> fix for DRILL-5082 and that increased the frequency of other known issues 
> (DRILL-5115 & DRILL-4832). Since there is no fix in-sight before the next 
> release (DRILL-1.11). I suggest we back off the original fix made for 
> DRILL-4831.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5098) Improving fault tolerance for connection between client and foreman node.

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5098?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5098:

Fix Version/s: (was: 1.10)
   1.10.0

> Improving fault tolerance for connection between client and foreman node.
> -
>
> Key: DRILL-5098
> URL: https://issues.apache.org/jira/browse/DRILL-5098
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Client - JDBC
>Reporter: Sorabh Hamirwasia
>Assignee: Sorabh Hamirwasia
>  Labels: doc-impacting, ready-to-commit
> Fix For: 1.10.0
>
>
> With DRILL-5015 we allowed support for specifying multiple Drillbits in 
> connection string and randomly choosing one out of it. Over time some of the 
> Drillbits specified in the connection string may die and the client can fail 
> to connect to Foreman node if random selection happens to be of dead Drillbit.
> Even if ZooKeeper is used for selecting a random Drillbit from the registered 
> one there is a small window when client selects one Drillbit and then that 
> Drillbit went down. The client will fail to connect to this Drillbit and 
> error out. 
> Instead if we try multiple Drillbits (configurable tries count through 
> connection string) then the probability of hitting this error window will 
> reduce in both the cases improving fault tolerance. During further 
> investigation it was also found that if there is Authentication failure then 
> we throw that error as generic RpcException. We need to improve that as well 
> to capture this case explicitly since in case of Auth failure we don't want 
> to try multiple Drillbits.
> Connection string example with new parameter:
> jdbc:drill:drillbit=[:][,[:]...;tries=5



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5207) Improve Parquet scan pipelining

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5207:

Fix Version/s: (was: 1.10)
   1.10.0

> Improve Parquet scan pipelining
> ---
>
> Key: DRILL-5207
> URL: https://issues.apache.org/jira/browse/DRILL-5207
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Storage - Parquet
>Affects Versions: 1.9.0
>Reporter: Parth Chandra
>Assignee: Parth Chandra
> Fix For: 1.10.0
>
>
> The parquet reader's async page reader is not quite efficiently pipelined. 
> The default size of the disk read buffer is 4MB while the page reader reads 
> ~1MB at a time. The Parquet decode is also processing 1MB at a time. This 
> means the disk is idle while the data is being processed. Reducing the buffer 
> to 1MB will reduce the time the processing thread waits for the disk read 
> thread.
> Additionally, since the data to process a page may be more or less than 1MB, 
> a queue of pages will help so that the disk scan does not block (until the 
> queue is full), waiting for the processing thread.
> Additionally, the BufferedDirectBufInputStream class reads from disk as soon 
> as it is initialized. Since this is called at setup time, this increases the 
> setup time for the query and query execution does not begin until this is 
> completed.
> There are a few other inefficiencies - options are read every time a page 
> reader is created. Reading options can be expensive.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5080) Create a memory-managed version of the External Sort operator

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5080?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5080:

Fix Version/s: (was: 1.10)
   1.10.0

> Create a memory-managed version of the External Sort operator
> -
>
> Key: DRILL-5080
> URL: https://issues.apache.org/jira/browse/DRILL-5080
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.10.0
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the external sort operator that 
> works to a clearly-defined memory limit. Attached is a design specification 
> for the work.
> The project will include fixing a number of bugs related to the external 
> sort, include as sub-tasks of this umbrella task.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5070) Code gen: create methods in fixed order to allow test verification

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua updated DRILL-5070:

Fix Version/s: (was: 1.10)
   1.10.0

> Code gen: create methods in fixed order to allow test verification
> --
>
> Key: DRILL-5070
> URL: https://issues.apache.org/jira/browse/DRILL-5070
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.8.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Minor
> Fix For: 1.10.0
>
>
> A handy technique in testing is to compare generated code against a "golden" 
> copy that defines the expected results. However, at present, Drill generates 
> code using the method order returned by {{Class.getDeclaredMethods}}, but 
> this method makes no guarantee about the order of the methods. The order 
> varies from one run to the next. There is some evidence [this 
> link|http://stackoverflow.com/questions/28585843/java-reflection-getdeclaredmethods-in-declared-order-strange-behaviour]
>  that order can vary even within a single run, though a quick test was unable 
> to reproduce this case.
> If method order does indeed vary within a single run, then the order can 
> impact the Drill code cache since it compares the sources from two different 
> generation events to detect duplicate code.
> This issue appeared when attempting to modify tests to capture generated code 
> for comparison to future results. Even a simple generated case from 
> {{ExpressionTest.testBasicExpression()}} that generates {{if(true) then 1 
> else 0 end}} (all constants) produced methods in different orders on each 
> test run.
> The fix is simple, in the {{SignatureHolder}} constructor, sort methods by 
> name after retrieving them from the class. The sort ensures that method order 
> is deterministic. Fortunately, the number of methods is small, so the sort 
> step adds little cost.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (DRILL-5230) Translation of millisecond duration into hours is incorrect

2017-02-01 Thread Kunal Khatua (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5230?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kunal Khatua reassigned DRILL-5230:
---

 Assignee: (was: Kunal Khatua)
Flags: Important
Affects Version/s: 1.9.0
   Labels: easyfix  (was: )
Fix Version/s: 1.10.0

> Translation of millisecond duration into hours is incorrect
> ---
>
> Key: DRILL-5230
> URL: https://issues.apache.org/jira/browse/DRILL-5230
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Web Server
>Affects Versions: 1.9.0
>Reporter: Kunal Khatua
>  Labels: easyfix
> Fix For: 1.10.0
>
>   Original Estimate: 24h
>  Remaining Estimate: 24h
>
> The method 
> {code:JAVA}org.apache.drill.exec.server.rest.profile.TableBuilder.appendMillis(long,
>  String){code}
> has a bug where the human readable translation of a 1+ hr duration in 
> milliseconds is reported incorrectly. 
> This has to do with the {code:JAVA}SimpleDateFormat.format() {code} method 
> incorectly translating it. 
> For e.g.
> {code:JAVA}
> long x = 4545342L; //1 hour 15 min 45.342 sec
> public void appendMillis(x, null);
> {code}
> This formats the value as {noformat}17h15m{noformat} instead of 
> {noformat}1h15m{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches for the first time

2017-02-01 Thread Zelaine Fong (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zelaine Fong updated DRILL-5237:

Labels: ready-to-commit  (was: )

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches for the first time
> 
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
>  Labels: ready-to-commit
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5215) CTTAS: disallow temp tables in view expansion logic

2017-02-01 Thread Zelaine Fong (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zelaine Fong updated DRILL-5215:

Labels: ready-to-commit  (was: )

> CTTAS: disallow temp tables in view expansion logic
> ---
>
> Key: DRILL-5215
> URL: https://issues.apache.org/jira/browse/DRILL-5215
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.10.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
>  Labels: ready-to-commit
> Fix For: 1.10.0
>
>
> This Jira is addressing last two CR comments in PR for Jira DRILL-4956:
> 1. Disallow temp tables in view expansion logic.
> Steps to reproduce:
> a. use default temporary workspace.
> b. create persistent table with name t:
> c. create view over persistent table.
> d. drop persistent table.
> e. create temporary table with the same name as persistent table.
> f. select from view 
> Currently Drill returns result on step f but should return error message.
> {noformat}
> VALIDATION ERROR: Temporary tables usage is disallowed. Used temporary table 
> name: t
> {noformat}
> {noformat}
> use dfs.tmp;
> create table t as select 'TABLE' from (values(1));
> create view v as select * from t;
> select * from v;
> +-+
> | EXPR$0  |
> +-+
> | TABLE   |
> +-+
> drop table t;
> create temporary table t as select 'TEMP' from (values(1));
> select * from v;
> +-+
> | EXPR$0  |
> +-+
> | TEMP|
> +-+
> {noformat}
> 2. Replace link to gist with CTTAS design doc to Jira link.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches for the first time

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user amansinha100 commented on the issue:

https://github.com/apache/drill/pull/735
  
lgtm.  +1


> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches for the first time
> 
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5215) CTTAS: disallow temp tables in view expansion logic

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/725
  
+1

LGTM


> CTTAS: disallow temp tables in view expansion logic
> ---
>
> Key: DRILL-5215
> URL: https://issues.apache.org/jira/browse/DRILL-5215
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.10.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Fix For: 1.10.0
>
>
> This Jira is addressing last two CR comments in PR for Jira DRILL-4956:
> 1. Disallow temp tables in view expansion logic.
> Steps to reproduce:
> a. use default temporary workspace.
> b. create persistent table with name t:
> c. create view over persistent table.
> d. drop persistent table.
> e. create temporary table with the same name as persistent table.
> f. select from view 
> Currently Drill returns result on step f but should return error message.
> {noformat}
> VALIDATION ERROR: Temporary tables usage is disallowed. Used temporary table 
> name: t
> {noformat}
> {noformat}
> use dfs.tmp;
> create table t as select 'TABLE' from (values(1));
> create view v as select * from t;
> select * from v;
> +-+
> | EXPR$0  |
> +-+
> | TABLE   |
> +-+
> drop table t;
> create temporary table t as select 'TEMP' from (values(1));
> select * from v;
> +-+
> | EXPR$0  |
> +-+
> | TEMP|
> +-+
> {noformat}
> 2. Replace link to gist with CTTAS design doc to Jira link.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5235) Column alias doubles sort data size when reading a text file

2017-02-01 Thread Zelaine Fong (JIRA)

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

Zelaine Fong commented on DRILL-5235:
-

If I do:

explain plan for select columns[0] col1 from `employee.json` order by col1;

this is what I get:

{code}
00-00Screen
00-01  Project(col1=[$0])
00-02SelectionVectorRemover
00-03  Sort(sort0=[$0], dir0=[ASC])
00-04Project(col1=[ITEM($0, 0)])
00-05  Scan(groupscan=[EasyGroupScan 
[selectionRoot=classpath:/employee.json, numFiles=1, columns=[`columns`[0]], 
files=[classpath:/employee.json]]])
{code}

I.e., there's only a single column in the project, and a single column in the 
sort.

> Column alias doubles sort data size when reading a text file
> 
>
> Key: DRILL-5235
> URL: https://issues.apache.org/jira/browse/DRILL-5235
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.9.0
>Reporter: Paul Rogers
>Priority: Minor
>
> Consider a simple query that reads data from a pipe-separated-value file and 
> sorts it. The file has just one column. The query looks something like this:
> {code}
> SELECT columns[0] col1 FROM `dfs.data`.`input-file.tbl` ORDER BY col1
> {code}
> Looking at the query plan, we see that a project operator not just creates an 
> alias {{col1}} for {{column\[0]}}, it also makes a *copy*.
> The particular input file is 20 GB in size and contains just one column. As a 
> result of materializing the alias, data size to the sort doubles to 40 GB. 
> This results in doubling query run time. If the sort must spill to disk, run 
> times increases by a much larger factor.
> The fix is to treat the alias as an alias, not a materialized copy.
> {code}
> {
>   "graph" : [ {
> "pop" : "fs-scan",
> "columns" : [ "`columns`[0]" ],
>   }, {
> "pop" : "project",
> "@id" : 4,
> "exprs" : [ {
>   "ref" : "`col1`",
>   "expr" : "`columns`[0]"
> } ],
>   }, {
> "pop" : "external-sort",
> "orderings" : [ {
>   "order" : "ASC",
>   "expr" : "`col1`",
>   "nullDirection" : "UNSPECIFIED"
> } ],
>   }, {
> "pop" : "selection-vector-remover",
>   }, {
> "pop" : "project",
>   }, {
> "pop" : "screen",
>   } ]
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5179) Dropping NULL Columns from putting in parquet files

2017-02-01 Thread mehran (JIRA)

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

mehran commented on DRILL-5179:
---

Is there any comments on this issue!
This issue will greatly helps us to test drill and I think having null columns 
is against drill philosophy.


> Dropping NULL Columns from putting in parquet files
> ---
>
> Key: DRILL-5179
> URL: https://issues.apache.org/jira/browse/DRILL-5179
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: mehran
>
> Due to schemaless nature of drill and parquet files. It is suitable that all 
> NULL columns be dropped in "CREATE TABLE" command. This will enhance speed of 
> header interpretation. I think this is a simple but big enhancement. 
> This can be an option in configuration parameters.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches for the first time

2017-02-01 Thread Zelaine Fong (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zelaine Fong updated DRILL-5237:

Reviewer: Aman Sinha

Assigned Reviewer to [~amansinha100]

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches for the first time
> 
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches for the first time

2017-02-01 Thread Serhii Harnyk (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Serhii Harnyk updated DRILL-5237:
-
Summary: FlattenRecordBatch loses nested fields from the schema when 
returns empty batches for the first time  (was: FlattenRecordBatch loses nested 
fields from the schema when returns empty batches at first time)

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches for the first time
> 
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches for the first time

2017-02-01 Thread Serhii Harnyk (JIRA)

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

Serhii Harnyk edited comment on DRILL-5237 at 2/1/17 12:56 PM:
---

This bug caused that FlattenRecordBatch loses nested fields from the schema 
when returns empty batches for the first time. When outer batches use these 
nested fields, they generate classes which operate with such fields like with 
constants. When next time FlattenRecordBatch returns non empty batch, generated 
classes of outer batches stay the same, so they returns wrong result.


was (Author: sharnyk):
This bug caused that FlattenRecordBatch loses nested fields from the schema 
when returns empty batches at first time. When outer batches use these nested 
fields, they generate classes which operate with such fields like with 
constants. When next time FlattenRecordBatch returns non empty batch, generated 
classes of outer batches stay the same, so they returns wrong result.

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches for the first time
> 
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches at first time

2017-02-01 Thread Serhii Harnyk (JIRA)

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

Serhii Harnyk commented on DRILL-5237:
--

This bug caused that FlattenRecordBatch loses nested fields from the schema 
when returns empty batches at first time. When outer batches use these nested 
fields, they generate classes which operate with such fields like with 
constants. When next time FlattenRecordBatch returns non empty batch, generated 
classes of outer batches stay the same, so they returns wrong result.

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches at first time
> ---
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5237) FlattenRecordBatch loses nested fields from the schema when returns empty batches at first time

2017-02-01 Thread Serhii Harnyk (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Serhii Harnyk updated DRILL-5237:
-
Summary: FlattenRecordBatch loses nested fields from the schema when 
returns empty batches at first time  (was: Same query produces different/wrong 
results)

> FlattenRecordBatch loses nested fields from the schema when returns empty 
> batches at first time
> ---
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5043) Function that returns a unique id per session/connection similar to MySQL's CONNECTION_ID()

2017-02-01 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5043:

Fix Version/s: (was: Future)
   1.10.0

> Function that returns a unique id per session/connection similar to MySQL's 
> CONNECTION_ID()
> ---
>
> Key: DRILL-5043
> URL: https://issues.apache.org/jira/browse/DRILL-5043
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Affects Versions: 1.8.0
>Reporter: Nagarajan Chinnasamy
>Priority: Minor
>  Labels: CONNECTION_ID, SESSION, UDF, doc-impacting, 
> ready-to-commit
> Fix For: 1.10.0
>
> Attachments: 01_session_id_sqlline.png, 
> 02_session_id_webconsole_query.png, 03_session_id_webconsole_result.png
>
>
> Design and implement a function that returns a unique id per 
> session/connection similar to MySQL's CONNECTION_ID().
> *Implementation details*
> function *session_id* will be added. Function returns current session unique 
> id represented as string. Parameter {code:java} boolean isNiladic{code} will 
> be added to UDF FunctionTemplate to indicate if a function is niladic (a 
> function to be called without any parameters and parentheses)
> Please note, this function will override columns that have the same name. 
> Table alias should be used to retrieve column value from table.
> Example:
> {code:sql}select session_id from   // returns the value of niladic 
> function session_id {code} 
> {code:sql}select t1.session_id from  t1 // returns session_id column 
> value from table {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (DRILL-5237) Same query produces different/wrong results

2017-02-01 Thread ASF GitHub Bot (JIRA)

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

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

GitHub user Serhii-Harnyk opened a pull request:

https://github.com/apache/drill/pull/735

DRILL-5237: Same query produces different/wrong results

Fixed loss of nested fields from the schema when the first 
FlattenRecordBatch is empty

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/Serhii-Harnyk/drill DRILL-5237

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/735.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #735


commit 800953b7ec924ea31ffc21327950c84528f3c492
Author: Serhii-Harnyk 
Date:   2017-01-27T15:36:10Z

DRILL-5237: Same query produces different/wrong results




> Same query produces different/wrong results
> ---
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (DRILL-5237) Same query produces different/wrong results

2017-02-01 Thread Serhii Harnyk (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Serhii Harnyk updated DRILL-5237:
-
Attachment: test_data.tar.gz

Files for reproducing this issue

> Same query produces different/wrong results
> ---
>
> Key: DRILL-5237
> URL: https://issues.apache.org/jira/browse/DRILL-5237
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Serhii Harnyk
>Assignee: Serhii Harnyk
> Attachments: test_data.tar.gz
>
>
> Query 
> {code:sql}
> select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as 
> b from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 
> 'a1');
> {code}
> returns different results for different values for option 
> planner.width.max_per_node
> With options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 1;
> {code}
> query returns correct result:
> {noformat}
> +--+
> | col  |
> +--+
> | 3|
> +--+
> {noformat}
> but with options 
> {code:sql}
> alter session set `planner.slice_target` = 1;
> alter session set `planner.width.max_per_node` = 3;
> {code}
> the same query returns wrong result 
> {noformat}
> +--+
> | col  |
> +--+
> | 2|
> +--+
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (DRILL-5237) Same query produces different/wrong results

2017-02-01 Thread Serhii Harnyk (JIRA)
Serhii Harnyk created DRILL-5237:


 Summary: Same query produces different/wrong results
 Key: DRILL-5237
 URL: https://issues.apache.org/jira/browse/DRILL-5237
 Project: Apache Drill
  Issue Type: Bug
Reporter: Serhii Harnyk
Assignee: Serhii Harnyk


Query 
{code:sql}
select count(*) as col from (select t1.a.a1 from (select t.*, flatten(t.b) as b 
from dfs.`/tmp/test_data` t where t.c is not null) t1 where t1.a .a1 like 'a1');
{code}
returns different results for different values for option 
planner.width.max_per_node
With options 
{code:sql}
alter session set `planner.slice_target` = 1;
alter session set `planner.width.max_per_node` = 1;
{code}
query returns correct result:
{noformat}
+--+
| col  |
+--+
| 3|
+--+
{noformat}
but with options 
{code:sql}
alter session set `planner.slice_target` = 1;
alter session set `planner.width.max_per_node` = 3;
{code}
the same query returns wrong result 
{noformat}
+--+
| col  |
+--+
| 2|
+--+
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)