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

2017-03-28 Thread Rahul Challapalli (JIRA)

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

Rahul Challapalli commented on DRILL-5080:
--

[~Paul.Rogers] Since the sub-tasks are marked as resolved for 1.11.0, would it 
make sense to also change the fix version for this umbrella jira? On the other 
hand we did create a memory managed external sort. Its just not enabled by 
default in 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
>  Labels: ready-to-commit
> 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-03-20 Thread Paul Rogers (JIRA)

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

Paul Rogers commented on DRILL-5080:


That is what Rahul has been doing for the last month or so.

Also, we are in the process of offering a PR for an internal test framework. 
Once that is done, we can do a PR for detailed "sub-operator" unit tests to 
verify many of the code-level bugs described here.

In general, we do need a good way to verify such internal bugs: QA can't do it 
from outside of the product because doing so is like fixing a watch wearing 
oven mitts...

> 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
>  Labels: ready-to-commit
> 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-03-20 Thread Kunal Khatua (JIRA)

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

Kunal Khatua commented on DRILL-5080:
-

[~paul-rogers], [~rkins] Do we need to create explicit tests for this ?

> 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
>  Labels: ready-to-commit
> 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-13 Thread ASF GitHub Bot (JIRA)

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

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

Github user asfgit closed the pull request at:

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


> 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
>  Labels: ready-to-commit
> 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-13 Thread ASF GitHub Bot (JIRA)

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

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

Github user amansinha100 commented on the issue:

https://github.com/apache/drill/pull/717
  
LGTM.  +1.  Thanks for the detailed writeup, comments and the responses to 
code review comments. 


> 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
>  Labels: ready-to-commit
> 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-12 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100710124
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100708912
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100711041
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100708601
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100710188
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100708660
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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

Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100703575
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100703657
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100686638
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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

Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100703843
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
 ---
@@ -0,0 +1,1456 @@
+/*
+ * 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.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+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.
+ * 
+ * 
+ * Data is spilled to disk as a "run". A run consists of one or more 
(typically
+ * many) batches, each of which is itself a sorted run of records.
+ * 
+ * During the sort/merge phase:
+ * 
+ * 
+ * When the input operator 

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

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

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

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

Github user amansinha100 commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r100705782
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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_r100657733
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/RecordBatchSizer.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.spill;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Given a record batch or vector container, determines the actual memory
+ * consumed by each column, the average row, and the entire record batch.
+ */
+
+public class RecordBatchSizer {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(RecordBatchSizer.class);
+
+  /**
+   * Column size information.
+   */
+  public static class ColumnSize {
+public final MaterializedField metadata;
+
+/**
+ * Assumed size from Drill metadata.
+ */
+public int stdSize;
+/**
+ * Actual memory consumed by all the vectors associated with this 
column.
+ */
+public int totalSize;
+/**
+ * Actual average column width as determined from actual memory use. 
This
+ * size is larger than the actual data size since this size includes 
per-
+ * column overhead such as any unused vector space, etc.
+ */
+public int estSize;
+
+/**
+ * The size of the data vector backing the column. Useful for detecting
+ * cases of possible direct memory fragmentation.
+ */
+public int dataVectorSize;
+public int capacity;
+public int density;
+public int dataSize;
+
+@SuppressWarnings("resource")
+public ColumnSize(VectorWrapper vw) {
+  metadata = vw.getField();
+  stdSize = TypeHelper.getSize(metadata.getType());
+
+  // Can't get size estimates if this is an empty batch.
+
+  ValueVector v = vw.getValueVector();
+  int rowCount = v.getAccessor().getValueCount();
+  if (rowCount == 0) {
+return;
+  }
+  DrillBuf[] bufs = v.getBuffers(false);
+  for (DrillBuf buf : bufs) {
+totalSize += buf.capacity();
+  }
+
+  // Capacity is the number of values that the vector could
+  // contain. This is useful only for fixed-length vectors.
+
+  capacity = v.getValueCapacity();
+
+  // Crude way to get the size of the buffer underlying simple 
(scalar) values.
+  // Ignores maps, lists and other esoterica. Uses a crude way to 
subtract out
+  // the null "bit" (really byte) buffer size for nullable vectors.
+
+  if (v instanceof BaseDataValueVector) {
+dataVectorSize = totalSize;
+if (v instanceof NullableVector) {
+  dataVectorSize -= bufs[0].getActualMemoryConsumed();
+}
+  }
+
+  // Determine "density" the number of rows 

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

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

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

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

Github user paul-rogers commented on the issue:

https://github.com/apache/drill/pull/717
  
Rebased and squashed commits to prepare for pulling into master.

Revised the code to estimate batch size. @Ben-Zvi, can you take a quick 
look?


> 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
>  Labels: ready-to-commit
> 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-08 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on the issue:

https://github.com/apache/drill/pull/717
  
Some comment got lost in the force-push. One was related to the output 
batch size, suggesting we cap it at 16 MB. The reason is that value vectors 
about 16 MB cause memory fragmentation. A later fix will limit an output batch 
to either 64K rows (the size of an sv2) or so that the longest vector is 
smaller than 16 MB. The most recent commit added per-column size information so 
that we can enforce this limit. For example, we can have 64K rows with columns 
of size 256 bytes within a 16 MB vector. There is no reason not to allow 64K 
rows even for rows with four of the 256 columns. Total batch size would be 64 
MB, but no single vector would be above 16 MB.

That fix will be offered, along with tests and enabling the managed sort by 
default, in a subsequent PR.




> 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
>  Labels: ready-to-commit
> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99246295
  
--- 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99245869
  
--- 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 --

Good catch! Fixed.


> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99247477
  
--- 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-5080) Create a memory-managed version of the External Sort operator

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99250103
  
--- 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 --

Here that is implied by the JSON-like syntax.


> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99247699
  
--- 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 --

Original code. Actually, these are no longer used, so removed them.


> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99246244
  
--- 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 --

Done.


> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99250032
  
--- 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 --

Good idea. Done.


> 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-06 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99246595
  
--- 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 

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

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

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

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

Github user paul-rogers commented on the issue:

https://github.com/apache/drill/pull/717
  
Rebased on latest master.


> 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-03 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on the issue:

https://github.com/apache/drill/pull/717
  
Build failure related to the slf4j library in the build process itself.


> 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99239035
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99238315
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99193890
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99197052
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99184602
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99237065
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99065219
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
--- End diff --

Fixed.


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

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99182115
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99067231
  
--- 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.
--- End diff --

Fixed.


> Create a memory-managed version of the 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99065262
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
--- End diff --

Fixed.


> Create a memory-managed version of the External Sort 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99067046
  
--- 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.
--- End diff --

We write spill files of size 300 MB. Any gains from long disk writes have 
already been gotten at that spill size. (That's 3 sec. of I/O for a spinning 
disk at 100 MB/s.)

Drill handles big data. A sort might be given 10 GB of memory to buffer 
batches. Spilling this entire amount, just because we need a small amount of 
room, does not make much sense. If we have 10 GB and read 11 GB of data, we'd 
want to write just four spill files of 300 MB each, then merge the in-memory 
and on-disk runs to avoid writing the other 10 GB.

This assumes, of course, that merging many in-batch runs is faster than 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99039840
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }
 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99066218
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99236335
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99235897
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99236749
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99065931
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99066331
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99065195
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
--- End diff --

Fixed


> 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 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99193167
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99066052
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99194009
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99065293
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99067830
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99234976
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99189185
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99194920
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99194543
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99037775
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }
 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99195103
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99181890
  
--- 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-02 Thread ASF GitHub Bot (JIRA)

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99040069
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }
 

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

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

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

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

Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/717#discussion_r99037413
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
@@ -63,13 +63,29 @@
   String SPOOLING_BUFFER_DELETE = "drill.exec.buffer.spooling.delete";
   String SPOOLING_BUFFER_MEMORY = "drill.exec.buffer.spooling.size";
   String BATCH_PURGE_THRESHOLD = "drill.exec.sort.purge.threshold";
+
+  // External Sort Boot configuration
+
   String EXTERNAL_SORT_TARGET_BATCH_SIZE = 
"drill.exec.sort.external.batch.size";
   String EXTERNAL_SORT_TARGET_SPILL_BATCH_SIZE = 
"drill.exec.sort.external.spill.batch.size";
   String EXTERNAL_SORT_SPILL_GROUP_SIZE = 
"drill.exec.sort.external.spill.group.size";
   String EXTERNAL_SORT_SPILL_THRESHOLD = 
"drill.exec.sort.external.spill.threshold";
   String EXTERNAL_SORT_SPILL_DIRS = 
"drill.exec.sort.external.spill.directories";
   String EXTERNAL_SORT_SPILL_FILESYSTEM = 
"drill.exec.sort.external.spill.fs";
+  String EXTERNAL_SORT_SPILL_FILE_SIZE = 
"drill.exec.sort.external.spill.file_size";
   String EXTERNAL_SORT_MSORT_MAX_BATCHSIZE = 
"drill.exec.sort.external.msort.batch.maxsize";
+  String EXTERNAL_SORT_DISABLE_MANAGED = 
"drill.exec.sort.external.disable_managed";
+  String EXTERNAL_SORT_MERGE_LIMIT = 
"drill.exec.sort.external.merge_limit";
+  String EXTERNAL_SORT_MIN_SPILL = 
"drill.exec.sort.external.spill.min_batches";
+  String EXTERNAL_SORT_MAX_SPILL = 
"drill.exec.sort.external.spill.max_batches";
+  String EXTERNAL_SORT_MAX_MEMORY = "drill.exec.sort.external.mem_limit";
+  String EXTERNAL_SORT_BATCH_LIMIT = 
"drill.exec.sort.external.batch_limit";
+
+  // External Sort Runtime options
+
+  BooleanValidator EXTERNAL_SORT_DISABLE_MANAGED_OPTION = new 
BooleanValidator("exec.sort.disable_managed", false);
--- End diff --

True. In the 1.10 release, this will change to "true". But, to prevent 
instability in this initial checkin, the old sort is left as the default. Of 
course, this PR has taken a while, so the new PR to set this to "true" may come 
only a few days after this PR is finally accepted...


> 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=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-5080) Create a memory-managed version of the External Sort operator

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

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

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_r98758075
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98793305
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98799527
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98590036
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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_r98818604
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98818785
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98594425
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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_r98589905
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
--- End diff --

There is no "allocator" among the parameters 


> Create a 

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

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

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

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_r98592329
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
+   * prepare it for a new collection of batches.
+   *
+   * @param batch the (hyper) batch of vectors to be copied
+   * @param batchGroupList same batches as above, but represented as a list
+   * of individual batches
+   * @param outputContainer the container into which to copy the batches
+   * @param allocator allocator to use to allocate memory in the operation
+   */
+
+  @SuppressWarnings("unchecked")
+  private void 

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

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

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

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_r98589289
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
+   * created, generated code and create it. If it has been created, close 
it and
--- End diff --

 "generate" ...


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

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

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

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

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_r95690405
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
 ---
@@ -0,0 +1,273 @@
+/*
+ * 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-5080) Create a memory-managed version of the External Sort operator

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

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

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_r98788215
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r95690247
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
 ---
@@ -0,0 +1,273 @@
+/*
+ * 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-5080) Create a memory-managed version of the External Sort operator

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

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

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_r98790875
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98768823
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98582657
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }

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

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

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

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_r98785941
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98584619
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }

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

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

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

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_r98824539
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98791132
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98802888
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98821658
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98589228
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import 
org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Manages a {@link PriorityQueueCopier} instance produced from code 
generation.
+ * Provides a wrapper around a copier "session" to simplify reading batches
+ * from the copier.
+ */
+
+public class CopierHolder {
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(CopierHolder.class);
+
+  private PriorityQueueCopier copier;
+
+  private final FragmentContext context;
+  private final BufferAllocator allocator;
+  private OperatorCodeGenerator opCodeGen;
+
+  public CopierHolder(FragmentContext context, BufferAllocator allocator, 
OperatorCodeGenerator opCodeGen) {
+this.context = context;
+this.allocator = allocator;
+this.opCodeGen = opCodeGen;
+  }
+
+  /**
+   * Start a merge operation using a temporary vector container. Used for
+   * intermediate merges.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param targetRecordCount
+   * @return
+   */
+
+  public CopierHolder.BatchMerger startMerge(BatchSchema schema, List batchGroupList, int targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, 
targetRecordCount);
+  }
+
+  /**
+   * Start a merge operation using the specified vector container. Used for
+   * the final merge operation.
+   *
+   * @param schema
+   * @param batchGroupList
+   * @param outputContainer
+   * @param targetRecordCount
+   * @return
+   */
+  public CopierHolder.BatchMerger startFinalMerge(BatchSchema schema, 
List batchGroupList, VectorContainer outputContainer, int 
targetRecordCount) {
+return new BatchMerger(this, schema, batchGroupList, outputContainer, 
targetRecordCount);
+  }
+
+  /**
+   * Prepare a copier which will write a collection of vectors to disk. 
The copier
+   * uses generated code to to the actual writes. If the copier has not 
yet been
--- End diff --

 "t" --> "d" 


> 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
>
> Attachments: ManagedExternalSortDesign.pdf
>
>
> We propose to create a "managed" version of the 

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

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

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

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_r98793059
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r95690449
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
 ---
@@ -0,0 +1,273 @@
+/*
+ * 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-5080) Create a memory-managed version of the External Sort operator

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

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

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_r98822217
  
--- 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-01-31 Thread ASF GitHub Bot (JIRA)

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

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_r98584176
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
 ---
@@ -0,0 +1,334 @@
+/*
+ * 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.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessible;
+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 com.google.common.base.Stopwatch;
+
+/**
+ * Represents a group of batches spilled to disk.
+ * 
+ * The batches are defined by a schema which can change over time. When 
the schema changes,
+ * all existing and new batches are coerced into the new schema. Provides a
+ * uniform way to iterate over records for one or more batches whether
+ * the batches are in memory or on disk.
+ * 
+ * The BatchGroup operates in two modes as given by the two
+ * subclasses:
+ * 
+ * Input mode (@link InputBatchGroup): Used to buffer in-memory batches
+ * prior to spilling.
+ * Spill mode (@link SpilledBatchGroup): Holds a "memento" to a set
+ * of batches written to disk. Acts as both a reader and writer for
+ * those batches.
+ */
+
+public abstract class BatchGroup implements VectorAccessible, 
AutoCloseable {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchGroup.class);
+
+  /**
+   * The input batch group gathers batches buffered in memory before
+   * spilling. The structure of the data is:
+   * 
+   * Contains a single batch received from the upstream (input)
+   * operator.
+   * Associated selection vector that provides a sorted
+   * indirection to the values in the batch.
+   * 
+   */
+
+  public static class InputBatch extends BatchGroup {
+private SelectionVector2 sv2;
+
+public InputBatch(VectorContainer container, SelectionVector2 sv2, 
OperatorContext context, long batchSize) {
+  super(container, context, batchSize);
+  this.sv2 = sv2;
+}
+
+public SelectionVector2 getSv2() {
+  return sv2;
+}
+
+@Override
+public int getRecordCount() {
+  if (sv2 != null) {
+return sv2.getCount();
+  } else {
+return super.getRecordCount();
+  }
+}
+
+@Override
+public int getNextIndex() {
+  int val = super.getNextIndex();
+  if (val == -1) {
+return val;
+  }
+  return sv2.getIndex(val);
+}
+
+@Override
+public void close() throws IOException {
+  try {
+super.close();
+  }
+  finally {
+if (sv2 != null) {
+  sv2.clear();
+}
+  }

  1   2   >