You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2017/06/21 18:29:09 UTC

[1/5] drill git commit: DRILL-5325: Unit tests for the managed sort

Repository: drill
Updated Branches:
  refs/heads/master c16e5f807 -> 90f43bff7


http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
new file mode 100644
index 0000000..e249c19
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -0,0 +1,609 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.OperatorFixture.OperatorFixtureBuilder;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.HyperRowSetImpl;
+import org.apache.drill.test.rowSet.IndirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSetBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+
+/**
+ * Tests the external sort implementation: the "guts" of the sort stripped of the
+ * Volcano-protocol layer. Assumes the individual components are already tested.
+ */
+
+public class TestSortImpl extends DrillTest {
+
+  /**
+   * Create the sort implementation to be used by test.
+   *
+   * @param fixture operator fixture
+   * @param sortOrder sort order as specified by {@link Ordering}
+   * @param nullOrder null order as specified by {@link Ordering}
+   * @param outputBatch where the sort should write its output
+   * @return the sort initialized sort implementation, ready to
+   * do work
+   */
+
+  public static SortImpl makeSortImpl(OperatorFixture fixture,
+                               String sortOrder, String nullOrder,
+                               VectorContainer outputBatch) {
+    FieldReference expr = FieldReference.getWithQuotedRef("key");
+    Ordering ordering = new Ordering(sortOrder, expr, nullOrder);
+    Sort popConfig = new Sort(null, Lists.newArrayList(ordering), false);
+    OperExecContext opContext = fixture.newOperExecContext(popConfig);
+    QueryId queryId = QueryId.newBuilder()
+        .setPart1(1234)
+        .setPart2(5678)
+        .build();
+    FragmentHandle handle = FragmentHandle.newBuilder()
+          .setMajorFragmentId(2)
+          .setMinorFragmentId(3)
+          .setQueryId(queryId)
+          .build();
+    SortConfig sortConfig = new SortConfig(opContext.getConfig());
+    SpillSet spillSet = new SpillSet(opContext.getConfig(), handle,
+                                     popConfig);
+    PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(opContext);
+    SpilledRuns spilledRuns = new SpilledRuns(opContext, spillSet, copierHolder);
+    return new SortImpl(opContext, sortConfig, spilledRuns, outputBatch);
+  }
+
+  /**
+   * Handy fixture to hold a sort, a set of input row sets (batches) and the
+   * output set of row sets (batches.) Pumps the input into the sort and
+   * harvests the output. Subclasses define the specifics of the sort,
+   * define the input data, and validate the output data.
+   */
+
+  public static class SortTestFixture {
+    private final OperatorFixture fixture;
+    private final List<RowSet> inputSets = new ArrayList<>();
+    private final List<RowSet> expected = new ArrayList<>();
+    String sortOrder = Ordering.ORDER_ASC;
+    String nullOrder = Ordering.NULLS_UNSPECIFIED;
+
+    public SortTestFixture(OperatorFixture fixture) {
+      this.fixture = fixture;
+    }
+
+    public SortTestFixture(OperatorFixture fixture, String sortOrder, String nullOrder) {
+      this.fixture = fixture;
+      this.sortOrder = sortOrder;
+      this.nullOrder = nullOrder;
+    }
+
+    public void addInput(RowSet input) {
+      inputSets.add(input);
+    }
+
+    public void addOutput(RowSet output) {
+      expected.add(output);
+    }
+
+    public void run() {
+      VectorContainer dest = new VectorContainer();
+      SortImpl sort = makeSortImpl(fixture, sortOrder, nullOrder, dest);
+
+      // Simulates a NEW_SCHEMA event
+
+      if (! inputSets.isEmpty()) {
+        sort.setSchema(inputSets.get(0).container().getSchema());
+      }
+
+      // Simulates an OK event
+
+      for (RowSet input : inputSets) {
+        sort.addBatch(input.vectorAccessible());
+      }
+
+      // Simulate returning results
+
+      SortResults results = sort.startMerge();
+      if (results.getContainer() != dest) {
+        dest.clear();
+        dest = results.getContainer();
+      }
+      for (RowSet expectedSet : expected) {
+        assertTrue(results.next());
+        RowSet rowSet = toRowSet(fixture, results, dest);
+        // Uncomment these for debugging. Leave them commented otherwise
+        // to avoid polluting the Maven build output unnecessarily.
+//        System.out.println("Expected:");
+//        expectedSet.print();
+//        System.out.println("Actual:");
+//        rowSet.print();
+        new RowSetComparison(expectedSet)
+              .verify(rowSet);
+        expectedSet.clear();
+      }
+      assertFalse(results.next());
+      validateSort(sort);
+      results.close();
+      dest.clear();
+      sort.close();
+      validateFinalStats(sort);
+    }
+
+    protected void validateSort(SortImpl sort) { }
+    protected void validateFinalStats(SortImpl sort) { }
+  }
+
+  /**
+   * Sort produces a variety of output types. Convert each type to the corresponding
+   * row set format. For historical reasons, the sort dumps its output into a vector
+   * container (normally attached to the external sort batch, here used stand-alone.)
+   *
+   * @param fixture operator test fixture
+   * @param results sort results iterator
+   * @param dest container that holds the sort results
+   * @return
+   */
+
+  private static RowSet toRowSet(OperatorFixture fixture, SortResults results, VectorContainer dest) {
+    if (results.getSv4() != null) {
+      return new HyperRowSetImpl(fixture.allocator(), dest, results.getSv4());
+    } else if (results.getSv2() != null) {
+      return new IndirectRowSet(fixture.allocator(), dest, results.getSv2());
+    } else {
+      return new DirectRowSet(fixture.allocator(), dest);
+    }
+  }
+
+  /**
+   * Test for null input (no input batches). Note that, in this case,
+   * we never see a schema.
+   * @throws Exception
+   */
+
+  @Test
+  public void testNullInput() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      SortTestFixture sortTest = new SortTestFixture(fixture);
+      sortTest.run();
+    }
+  }
+
+  /**
+   * Test for an input with a schema, but only an empty input batch.
+   * @throws Exception
+   */
+
+  @Test
+  public void testEmptyInput() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      BatchSchema schema = SortTestUtilities.nonNullSchema();
+      SortTestFixture sortTest = new SortTestFixture(fixture);
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .build());
+      sortTest.run();
+    }
+  }
+
+  /**
+   * Degenerate case: single row in single batch.
+   * @throws Exception
+   */
+
+  @Test
+  public void testSingleRow() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      BatchSchema schema = SortTestUtilities.nonNullSchema();
+      SortTestFixture sortTest = new SortTestFixture(fixture);
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .build());
+      sortTest.addOutput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .build());
+      sortTest.run();
+    }
+  }
+
+  /**
+   * Degenerate case: two (unsorted) rows in single batch
+   * @throws Exception
+   */
+
+  @Test
+  public void testSingleBatch() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      BatchSchema schema = SortTestUtilities.nonNullSchema();
+      SortTestFixture sortTest = new SortTestFixture(fixture);
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(2, "second")
+          .add(1, "first")
+          .build());
+      sortTest.addOutput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .add(2, "second")
+          .build());
+      sortTest.run();
+    }
+  }
+
+  /**
+   * Degenerate case, one row in each of two
+   * (unsorted) batches.
+   * @throws Exception
+   */
+
+  @Test
+  public void testTwoBatches() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      BatchSchema schema = SortTestUtilities.nonNullSchema();
+      SortTestFixture sortTest = new SortTestFixture(fixture);
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(2, "second")
+          .build());
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .build());
+      sortTest.addOutput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .add(2, "second")
+          .build());
+      sortTest.run();
+    }
+  }
+
+  /**
+   * Crude-but-effective data generator that produces pseudo-random data
+   * that can be easily verified. The pseudo-random data is generate by the
+   * simple means of incrementing a counter using a random value, and wrapping.
+   * This ensures we visit each value twice, and that the sorted output will
+   * be a continuous run of numbers in proper order.
+   */
+
+  public static class DataGenerator {
+    private final OperatorFixture fixture;
+    private final BatchSchema schema;
+    private final int targetCount;
+    private final int batchSize;
+    private final int step;
+    private int rowCount;
+    private int currentValue;
+
+    public DataGenerator(OperatorFixture fixture, int targetCount, int batchSize) {
+      this(fixture, targetCount, batchSize, 0, guessStep(targetCount));
+    }
+
+    public DataGenerator(OperatorFixture fixture, int targetCount, int batchSize, int seed, int step) {
+      this.fixture = fixture;
+      this.targetCount = targetCount;
+      this.batchSize = Math.min(batchSize, Character.MAX_VALUE);
+      this.step = step;
+      schema = SortTestUtilities.nonNullSchema();
+      currentValue = seed;
+    }
+
+    /**
+     * Pick a reasonable prime step based on data size.
+     *
+     * @param target number of rows to generate
+     * @return the prime step size
+     */
+
+    private static int guessStep(int target) {
+      if (target < 10) {
+        return 7;
+      } else if (target < 200) {
+        return 71;
+      } else if (target < 2000) {
+        return 701;
+      } else if (target < 20000) {
+        return 7001;
+      } else {
+        return 17011;
+      }
+    }
+
+    public RowSet nextRowSet() {
+      if (rowCount == targetCount) {
+        return null;
+      }
+      RowSetBuilder builder = fixture.rowSetBuilder(schema);
+      int end = Math.min(batchSize, targetCount - rowCount);
+      for (int i = 0; i < end; i++) {
+        builder.add(currentValue, i + ", " + currentValue);
+        currentValue = (currentValue + step) % targetCount;
+        rowCount++;
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Validate a sort output batch based on the expectation that the key
+   * is an ordered sequence of integers, split across multiple batches.
+   */
+
+  public static class DataValidator {
+    private final int targetCount;
+    private final int batchSize;
+    private int batchCount;
+    private int rowCount;
+
+    public DataValidator(int targetCount, int batchSize) {
+      this.targetCount = targetCount;
+      this.batchSize = Math.min(batchSize, Character.MAX_VALUE);
+    }
+
+    public void validate(RowSet output) {
+      batchCount++;
+      int expectedSize = Math.min(batchSize, targetCount - rowCount);
+      assertEquals("Size of batch " + batchCount, expectedSize, output.rowCount());
+      RowSetReader reader = output.reader();
+      while (reader.next()) {
+        assertEquals("Value of " + batchCount + ":" + rowCount,
+            rowCount, reader.column(0).getInt());
+        rowCount++;
+      }
+    }
+
+    public void validateDone() {
+      assertEquals("Wrong row count", targetCount, rowCount);
+    }
+  }
+
+  Stopwatch timer = Stopwatch.createUnstarted();
+
+  /**
+   * Run a full-blown sort test with multiple input batches. Because we want to
+   * generate multiple inputs, we don't create them statically. Instead, we generate
+   * them on the fly using a data generator. A matching data validator verifies the
+   * output. Here, we are focusing on overall test flow. Separate, detailed, unit
+   * tests have already probed the details of each sort component and data type,
+   * so we don't need to repeat that whole exercise here; using integer keys is
+   * sufficient.
+   *
+   * @param fixture the operator test fixture
+   * @param dataGen input batch generator
+   * @param validator validates output batches
+   */
+
+  public void runLargeSortTest(OperatorFixture fixture, DataGenerator dataGen,
+                               DataValidator validator) {
+    VectorContainer dest = new VectorContainer();
+    SortImpl sort = makeSortImpl(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED, dest);
+
+    int batchCount = 0;
+    RowSet input;
+    while ((input = dataGen.nextRowSet()) != null) {
+      batchCount++;
+      if (batchCount == 1) {
+        // Simulates a NEW_SCHEMA event
+
+        timer.start();
+        sort.setSchema(input.container().getSchema());
+        timer.stop();
+      }
+
+      // Simulates an OK event
+
+      timer.start();
+      sort.addBatch(input.vectorAccessible());
+      timer.stop();
+    }
+
+    // Simulate returning results
+
+    timer.start();
+    SortResults results = sort.startMerge();
+    if (results.getContainer() != dest) {
+      dest.clear();
+      dest = results.getContainer();
+    }
+    while (results.next()) {
+      timer.stop();
+      RowSet output = toRowSet(fixture, results, dest);
+      validator.validate(output);
+      timer.start();
+    }
+    timer.stop();
+    validator.validateDone();
+    results.close();
+    dest.clear();
+    sort.close();
+  }
+
+  /**
+   * Set up and run a test for "jumbo" batches, and time the run.
+   * @param fixture operator test fixture
+   * @param rowCount number of rows to test
+   */
+
+  public void runJumboBatchTest(OperatorFixture fixture, int rowCount) {
+    timer.reset();
+    DataGenerator dataGen = new DataGenerator(fixture, rowCount, Character.MAX_VALUE);
+    DataValidator validator = new DataValidator(rowCount, Character.MAX_VALUE);
+    runLargeSortTest(fixture, dataGen, validator);
+    System.out.println(timer.elapsed(TimeUnit.MILLISECONDS));
+  }
+
+  /**
+   * Most tests have used small row counts because we want to probe specific bits
+   * of interest. Try 1000 rows just to ensure things work
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testModerateBatch() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      runJumboBatchTest(fixture, 1000);
+    }
+  }
+
+  /**
+   * Hit the sort with the largest possible batch size to ensure nothing is lost
+   * at the edges.
+   *
+   * @throws Exception
+   */
+
+  @Test
+  public void testLargeBatch() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      runJumboBatchTest(fixture, Character.MAX_VALUE);
+    }
+  }
+
+  /**
+   * Run a test using wide rows. This stresses the "copier" portion of the sort
+   * and allows us to test the original generated copier and the revised "generic"
+   * copier.
+   *
+   * @param fixture operator test fixture
+   * @param colCount number of data (non-key) columns
+   * @param rowCount number of rows to generate
+   */
+
+  public void runWideRowsTest(OperatorFixture fixture, int colCount, int rowCount) {
+    SchemaBuilder builder = new SchemaBuilder()
+        .add("key", MinorType.INT);
+    for (int i = 0; i < colCount; i++) {
+      builder.add("col" + (i+1), MinorType.INT);
+    }
+    BatchSchema schema = builder.build();
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer(rowCount);
+    for (int i = 0; i < rowCount; i++) {
+      writer.set(0, i);
+      for (int j = 0; j < colCount; j++) {
+        writer.set(j + 1, i * 100_000 + j);
+      }
+      writer.save();
+    }
+    writer.done();
+
+    VectorContainer dest = new VectorContainer();
+    SortImpl sort = makeSortImpl(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED, dest);
+    timer.reset();
+    timer.start();
+    sort.setSchema(rowSet.container().getSchema());
+    sort.addBatch(rowSet.vectorAccessible());
+    SortResults results = sort.startMerge();
+    if (results.getContainer() != dest) {
+      dest.clear();
+      dest = results.getContainer();
+    }
+    assertTrue(results.next());
+    timer.stop();
+    assertFalse(results.next());
+    results.close();
+    dest.clear();
+    sort.close();
+    System.out.println(timer.elapsed(TimeUnit.MILLISECONDS));
+  }
+
+  /**
+   * Test wide rows with the stock copier.
+   *
+   * @throws Exception
+   */
+
+  @Test
+  public void testWideRows() throws Exception {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture()) {
+      runWideRowsTest(fixture, 1000, Character.MAX_VALUE);
+    }
+  }
+
+  /**
+   * Force the sorter to spill, and verify that the resulting data
+   * is correct. Uses a specific property of the sort to set the
+   * in-memory batch limit so that we don't have to fiddle with filling
+   * up memory. The point here is not to test the code that decides when
+   * to spill (that was already tested.) Nor to test the spilling
+   * mechanism itself (that has also already been tested.) Rather it is
+   * to ensure that, when those components are integrated into the
+   * sort implementation, that the whole assembly does the right thing.
+   *
+   * @throws Exception
+   */
+
+  @Test
+  public void testSpill() throws Exception {
+    OperatorFixtureBuilder builder = OperatorFixture.builder();
+    builder.configBuilder()
+      .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 2);
+    try (OperatorFixture fixture = builder.build()) {
+      BatchSchema schema = SortTestUtilities.nonNullSchema();
+      SortTestFixture sortTest = new SortTestFixture(fixture) {
+        @Override
+        protected void validateSort(SortImpl sort) {
+          assertEquals(1, sort.getMetrics().getSpillCount());
+          assertEquals(0, sort.getMetrics().getMergeCount());
+          assertEquals(2, sort.getMetrics().getPeakBatchCount());
+        }
+        @Override
+        protected void validateFinalStats(SortImpl sort) {
+          assertTrue(sort.getMetrics().getWriteBytes() > 0);
+        }
+      };
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(2, "second")
+          .build());
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(3, "third")
+          .build());
+      sortTest.addInput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .build());
+      sortTest.addOutput(fixture.rowSetBuilder(schema)
+          .add(1, "first")
+          .add(2, "second")
+          .add(3, "third")
+          .build());
+      sortTest.run();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
new file mode 100644
index 0000000..dd371d7
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
@@ -0,0 +1,605 @@
+/*
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Random;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.joda.time.Period;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Tests the generated per-batch sort code via its wrapper layer.
+ */
+
+public class TestSorter extends DrillTest {
+
+  public static OperatorFixture fixture;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    fixture = OperatorFixture.builder().build();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    fixture.close();
+  }
+
+  public static Sort makeSortConfig(String key, String sortOrder, String nullOrder) {
+    FieldReference expr = FieldReference.getWithQuotedRef(key);
+    Ordering ordering = new Ordering(sortOrder, expr, nullOrder);
+    return new Sort(null, Lists.newArrayList(ordering), false);
+  }
+
+  public void runSorterTest(SingleRowSet rowSet, SingleRowSet expected) throws Exception {
+    runSorterTest(makeSortConfig("key", Ordering.ORDER_ASC, Ordering.NULLS_LAST), rowSet, expected);
+  }
+
+  public void runSorterTest(Sort popConfig, SingleRowSet rowSet, SingleRowSet expected) throws Exception {
+    OperExecContext opContext = fixture.newOperExecContext(popConfig);
+    SorterWrapper sorter = new SorterWrapper(opContext);
+
+    sorter.sortBatch(rowSet.container(), rowSet.getSv2());
+
+    new RowSetComparison(expected)
+        .verifyAndClear(rowSet);
+    sorter.close();
+  }
+
+  // Test degenerate case: no rows
+
+  @Test
+  public void testEmptyRowSet() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    SingleRowSet rowSet = new RowSetBuilder(fixture.allocator(), schema)
+        .withSv2()
+        .build();
+    SingleRowSet expected = new RowSetBuilder(fixture.allocator(), schema)
+        .build();
+    runSorterTest(rowSet, expected);
+  }
+
+  // Sanity test: single row
+
+  @Test
+  public void testSingleRow() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    SingleRowSet rowSet = new RowSetBuilder(fixture.allocator(), schema)
+          .add(0, "0")
+          .withSv2()
+          .build();
+
+    SingleRowSet expected = new RowSetBuilder(fixture.allocator(), schema)
+        .add(0, "0")
+        .build();
+    runSorterTest(rowSet, expected);
+  }
+
+  // Paranoia: sort with two rows.
+
+  @Test
+  public void testTwoRows() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    SingleRowSet rowSet = new RowSetBuilder(fixture.allocator(), schema)
+        .add(1, "1")
+        .add(0, "0")
+        .withSv2()
+        .build();
+
+    SingleRowSet expected = new RowSetBuilder(fixture.allocator(), schema)
+        .add(0, "0")
+        .add(1, "1")
+        .build();
+    runSorterTest(rowSet, expected);
+  }
+
+  private abstract static class BaseSortTester {
+    protected final OperatorFixture fixture;
+    protected final SorterWrapper sorter;
+    protected final boolean nullable;
+
+    public BaseSortTester(OperatorFixture fixture, String sortOrder, String nullOrder, boolean nullable) {
+      this.fixture = fixture;
+      Sort popConfig = makeSortConfig("key", sortOrder, nullOrder);
+      this.nullable = nullable;
+
+      OperExecContext opContext = fixture.newOperExecContext(popConfig);
+      sorter = new SorterWrapper(opContext);
+    }
+  }
+
+  private abstract static class SortTester extends BaseSortTester {
+
+    protected DataItem data[];
+
+    public SortTester(OperatorFixture fixture, String sortOrder, String nullOrder, boolean nullable) {
+      super(fixture, sortOrder, nullOrder, nullable);
+    }
+
+    public void test(MinorType type) throws SchemaChangeException {
+      data = makeDataArray(20);
+      BatchSchema schema = SortTestUtilities.makeSchema(type, nullable);
+      SingleRowSet input = makeDataSet(fixture.allocator(), schema, data);
+      input = input.toIndirect();
+      sorter.sortBatch(input.container(), input.getSv2());
+      sorter.close();
+      verify(input);
+    }
+
+    public static class DataItem {
+      public final int key;
+      public final int value;
+      public final boolean isNull;
+
+      public DataItem(int key, int value, boolean isNull) {
+        this.key = key;
+        this.value = value;
+        this.isNull = isNull;
+      }
+
+      @Override
+      public String toString() {
+        return "(" + key + ", \"" + value + "\", " +
+               (isNull ? "null" : "set") + ")";
+      }
+    }
+
+    public DataItem[] makeDataArray(int size) {
+      DataItem values[] = new DataItem[size];
+      int key = 11;
+      int delta = 3;
+      for (int i = 0; i < size; i++) {
+        values[i] = new DataItem(key, i, key % 5 == 0);
+        key = (key + delta) % size;
+      }
+      return values;
+    }
+
+    public SingleRowSet makeDataSet(BufferAllocator allocator, BatchSchema schema, DataItem[] items) {
+      ExtendableRowSet rowSet = fixture.rowSet(schema);
+      RowSetWriter writer = rowSet.writer(items.length);
+      for (int i = 0; i < items.length; i++) {
+        DataItem item = items[i];
+        if (nullable && item.isNull) {
+          writer.column(0).setNull();
+        } else {
+          RowSetUtilities.setFromInt(writer, 0, item.key);
+        }
+        writer.column(1).setString(Integer.toString(item.value));
+        writer.save();
+      }
+      writer.done();
+      return rowSet;
+    }
+
+    private void verify(RowSet actual) {
+      DataItem expected[] = Arrays.copyOf(data, data.length);
+      doSort(expected);
+      RowSet expectedRows = makeDataSet(actual.allocator(), actual.schema().batch(), expected);
+//      System.out.println("Expected:");
+//      expectedRows.print();
+//      System.out.println("Actual:");
+//      actual.print();
+      doVerify(expected, expectedRows, actual);
+    }
+
+    protected void doVerify(DataItem[] expected, RowSet expectedRows, RowSet actual) {
+      new RowSetComparison(expectedRows)
+            .verifyAndClear(actual);
+    }
+
+    protected abstract void doSort(DataItem[] expected);
+  }
+
+  private static class TestSorterNumeric extends SortTester {
+
+    private final int sign;
+
+    public TestSorterNumeric(OperatorFixture fixture, boolean asc) {
+      super(fixture,
+            asc ? Ordering.ORDER_ASC : Ordering.ORDER_DESC,
+            Ordering.NULLS_UNSPECIFIED, false);
+      sign = asc ? 1 : -1;
+    }
+
+    @Override
+    protected void doSort(DataItem[] expected) {
+      Arrays.sort(expected, new Comparator<DataItem>(){
+        @Override
+        public int compare(DataItem o1, DataItem o2) {
+          return sign * Integer.compare(o1.key, o2.key);
+        }
+      });
+    }
+  }
+
+  private static class TestSorterNullableNumeric extends SortTester {
+
+    private final int sign;
+    private final int nullSign;
+
+    public TestSorterNullableNumeric(OperatorFixture fixture, boolean asc, boolean nullsLast) {
+      super(fixture,
+          asc ? Ordering.ORDER_ASC : Ordering.ORDER_DESC,
+          nullsLast ? Ordering.NULLS_LAST : Ordering.NULLS_FIRST,
+          true);
+      sign = asc ? 1 : -1;
+      nullSign = nullsLast ? 1 : -1;
+    }
+
+    @Override
+    protected void doSort(DataItem[] expected) {
+      Arrays.sort(expected, new Comparator<DataItem>(){
+        @Override
+        public int compare(DataItem o1, DataItem o2) {
+          if (o1.isNull  &&  o2.isNull) { return 0; }
+          if (o1.isNull) { return nullSign; }
+          if (o2.isNull) { return -nullSign; }
+          return sign * Integer.compare(o1.key, o2.key);
+        }
+      });
+    }
+
+    @Override
+    protected void doVerify(DataItem[] expected, RowSet expectedRows, RowSet actual) {
+      int nullCount = 0;
+      for (DataItem item : expected) {
+        if (item.isNull) { nullCount++; }
+      }
+      int length = expected.length - nullCount;
+      int offset = (nullSign == 1) ? 0 : nullCount;
+      new RowSetComparison(expectedRows)
+            .offset(offset)
+            .span(length)
+            .verify(actual);
+      offset = length - offset;
+      new RowSetComparison(expectedRows)
+            .offset(offset)
+            .span(nullCount)
+            .withMask(true, false)
+            .verifyAndClear(actual);
+    }
+  }
+
+  private static class TestSorterStringAsc extends SortTester {
+
+    public TestSorterStringAsc(OperatorFixture fixture) {
+      super(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED, false);
+    }
+
+    @Override
+    protected void doSort(DataItem[] expected) {
+      Arrays.sort(expected, new Comparator<DataItem>(){
+        @Override
+        public int compare(DataItem o1, DataItem o2) {
+          return Integer.toString(o1.key).compareTo(Integer.toString(o2.key));
+        }
+      });
+    }
+  }
+
+  private static class TestSorterBinaryAsc extends SortTester {
+
+    public TestSorterBinaryAsc(OperatorFixture fixture) {
+      super(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED, false);
+    }
+
+    @Override
+    protected void doSort(DataItem[] expected) {
+      Arrays.sort(expected, new Comparator<DataItem>(){
+        @Override
+        public int compare(DataItem o1, DataItem o2) {
+          return Integer.toHexString(o1.key).compareTo(Integer.toHexString(o2.key));
+        }
+      });
+    }
+  }
+
+  private abstract static class BaseTestSorterIntervalAsc extends BaseSortTester {
+
+    public BaseTestSorterIntervalAsc(OperatorFixture fixture) {
+      super(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED, false);
+    }
+
+    public void test(MinorType type) throws SchemaChangeException {
+      BatchSchema schema = new SchemaBuilder()
+          .add("key", type)
+          .build();
+      SingleRowSet input = makeInputData(fixture.allocator(), schema);
+      input = input.toIndirect();
+      sorter.sortBatch(input.container(), input.getSv2());
+      sorter.close();
+      verify(input);
+      input.clear();
+    }
+
+    protected SingleRowSet makeInputData(BufferAllocator allocator,
+        BatchSchema schema) {
+      RowSetBuilder builder = fixture.rowSetBuilder(schema);
+      int rowCount = 100;
+      Random rand = new Random();
+      for (int i = 0; i < rowCount; i++) {
+        int ms = rand.nextInt(1000);
+        int sec = rand.nextInt(60);
+        int min = rand.nextInt(60);
+        int hr = rand.nextInt(24);
+        int day = rand.nextInt(28);
+        int mo = rand.nextInt(12);
+        int yr = rand.nextInt(10);
+        Period period = makePeriod(yr, mo, day, hr, min, sec, ms);
+         builder.add(period);
+      }
+      return builder.build();
+    }
+
+    protected abstract Period makePeriod(int yr, int mo, int day, int hr, int min, int sec,
+        int ms);
+
+    private void verify(SingleRowSet output) {
+      RowSetReader reader = output.reader();
+      int prevYears = 0;
+      int prevMonths = 0;
+      long prevMs = 0;
+      while (reader.next()) {
+        Period period = reader.column(0).getPeriod().normalizedStandard();
+//        System.out.println(period);
+        int years = period.getYears();
+        assertTrue(prevYears <= years);
+        if (prevYears != years) {
+          prevMonths = 0;
+          prevMs = 0;
+        }
+        prevYears = years;
+
+        int months = period.getMonths();
+        assertTrue(prevMonths <= months);
+        if (prevMonths != months) {
+           prevMs = 0;
+        }
+        prevMonths = months;
+
+        Period remainder = period
+            .withYears(0)
+            .withMonths(0);
+
+        long ms = remainder.toStandardDuration().getMillis();
+        assertTrue(prevMs <= ms);
+        prevMs = ms;
+      }
+    }
+  }
+
+  private static class TestSorterIntervalAsc extends BaseTestSorterIntervalAsc {
+
+    public TestSorterIntervalAsc(OperatorFixture fixture) {
+      super(fixture);
+    }
+
+    public void test() throws SchemaChangeException {
+      test(MinorType.INTERVAL);
+    }
+
+    @Override
+    protected Period makePeriod(int yr, int mo, int day, int hr, int min,
+        int sec, int ms) {
+      return Period.years(yr)
+          .withMonths(mo)
+          .withDays(day)
+          .withHours(hr)
+          .withMinutes(min)
+          .withSeconds(sec)
+          .withMillis(ms);
+    }
+  }
+
+  private static class TestSorterIntervalYearAsc extends BaseTestSorterIntervalAsc {
+
+    public TestSorterIntervalYearAsc(OperatorFixture fixture) {
+      super(fixture);
+    }
+
+    public void test() throws SchemaChangeException {
+      test(MinorType.INTERVALYEAR);
+    }
+
+    @Override
+    protected Period makePeriod(int yr, int mo, int day, int hr, int min,
+        int sec, int ms) {
+      return Period.years(yr)
+          .withMonths(mo);
+    }
+  }
+
+  private static class TestSorterIntervalDayAsc extends BaseTestSorterIntervalAsc {
+
+    public TestSorterIntervalDayAsc(OperatorFixture fixture) {
+      super(fixture);
+    }
+
+    public void test() throws SchemaChangeException {
+      test(MinorType.INTERVALDAY);
+    }
+
+    @Override
+    protected Period makePeriod(int yr, int mo, int day, int hr, int min,
+        int sec, int ms) {
+      return Period.days(day)
+          .withHours(hr)
+          .withMinutes(min)
+          .withSeconds(sec)
+          .withMillis(ms);
+    }
+  }
+
+  @Test
+  public void testNumericTypes() throws Exception {
+    TestSorterNumeric tester1 = new TestSorterNumeric(fixture, true);
+//      tester1.test(MinorType.TINYINT); // DRILL-5329
+//      tester1.test(MinorType.UINT1); DRILL-5329
+//      tester1.test(MinorType.SMALLINT); DRILL-5329
+//      tester1.test(MinorType.UINT2); DRILL-5329
+    tester1.test(MinorType.INT);
+//      tester1.test(MinorType.UINT4); DRILL-5329
+    tester1.test(MinorType.BIGINT);
+//      tester1.test(MinorType.UINT8); DRILL-5329
+    tester1.test(MinorType.FLOAT4);
+    tester1.test(MinorType.FLOAT8);
+    tester1.test(MinorType.DECIMAL9);
+    tester1.test(MinorType.DECIMAL18);
+//      tester1.test(MinorType.DECIMAL28SPARSE); DRILL-5329
+//      tester1.test(MinorType.DECIMAL38SPARSE); DRILL-5329
+//    tester1.test(MinorType.DECIMAL28DENSE); No writer
+//    tester1.test(MinorType.DECIMAL38DENSE); No writer
+    tester1.test(MinorType.DATE);
+    tester1.test(MinorType.TIME);
+    tester1.test(MinorType.TIMESTAMP);
+  }
+
+  @Test
+  public void testVarCharTypes() throws Exception {
+    TestSorterStringAsc tester = new TestSorterStringAsc(fixture);
+    tester.test(MinorType.VARCHAR);
+//      tester.test(MinorType.VAR16CHAR); DRILL-5329
+  }
+
+  /**
+   * Test the VARBINARY data type as a sort key.
+   *
+   * @throws Exception for internal errors
+   */
+
+  @Test
+  public void testVarBinary() throws Exception {
+    TestSorterBinaryAsc tester = new TestSorterBinaryAsc(fixture);
+    tester.test(MinorType.VARBINARY);
+  }
+
+  /**
+   * Test the INTERVAL data type as a sort key.
+   *
+   * @throws Exception for internal errors
+   */
+
+  @Test
+  public void testInterval() throws Exception {
+    TestSorterIntervalAsc tester = new TestSorterIntervalAsc(fixture);
+    tester.test();
+  }
+
+  /**
+   * Test the INTERVALYEAR data type as a sort key.
+   *
+   * @throws Exception for internal errors
+   */
+
+  @Test
+  public void testIntervalYear() throws Exception {
+    TestSorterIntervalYearAsc tester = new TestSorterIntervalYearAsc(fixture);
+    tester.test();
+  }
+
+  /**
+   * Test the INTERVALDAY data type as a sort key.
+   *
+   * @throws Exception for internal errors
+   */
+
+  @Test
+  public void testIntervalDay() throws Exception {
+    TestSorterIntervalDayAsc tester = new TestSorterIntervalDayAsc(fixture);
+    tester.test();
+  }
+
+  @Test
+  public void testDesc() throws Exception {
+    TestSorterNumeric tester = new TestSorterNumeric(fixture, false);
+    tester.test(MinorType.INT);
+  }
+
+  /**
+   * Verify that nulls sort in the requested position: high or low.
+   * Earlier tests verify that "unspecified" maps to high or low
+   * depending on sort order.
+   */
+
+  @Test
+  public void testNullable() throws Exception {
+    TestSorterNullableNumeric tester = new TestSorterNullableNumeric(fixture, true, true);
+    tester.test(MinorType.INT);
+    tester = new TestSorterNullableNumeric(fixture, true, false);
+    tester.test(MinorType.INT);
+    tester = new TestSorterNullableNumeric(fixture, false, true);
+    tester.test(MinorType.INT);
+    tester = new TestSorterNullableNumeric(fixture, false, false);
+    tester.test(MinorType.INT);
+  }
+
+  @Test
+  @Ignore("DRILL-5384")
+  public void testMapKey() throws Exception {
+    BatchSchema schema = new SchemaBuilder()
+        .addMap("map")
+          .add("key", MinorType.INT)
+          .add("value", MinorType.VARCHAR)
+          .buildMap()
+        .build();
+
+    SingleRowSet input = fixture.rowSetBuilder(schema)
+        .add(3, "third")
+        .add(1, "first")
+        .add(2, "second")
+        .withSv2()
+        .build();
+
+    SingleRowSet output = fixture.rowSetBuilder(schema)
+        .add(1, "first")
+        .add(2, "second")
+        .add(3, "third")
+        .build();
+    Sort popConfig = makeSortConfig("map.key", Ordering.ORDER_ASC, Ordering.NULLS_LAST);
+    runSorterTest(popConfig, input, output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
index d872d67..8efa3ee 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
@@ -50,6 +50,13 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
   public static final int DEBUG_LOG_LENGTH = 6;
   public static final boolean DEBUG = AssertionUtil.isAssertionsEnabled()
       || Boolean.parseBoolean(System.getProperty(DEBUG_ALLOCATOR, "false"));
+
+  /**
+   * Size of the I/O buffer used when writing to files. Set here
+   * because the buffer is used multiple times by an operator.
+   */
+
+  private static final int IO_BUFFER_SIZE = 32*1024;
   private final Object DEBUG_LOCK = DEBUG ? new Object() : null;
 
   private final BaseAllocator parentAllocator;
@@ -241,7 +248,6 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
         releaseBytes(actualRequestSize);
       }
     }
-
   }
 
   /**
@@ -452,7 +458,6 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
         historicalLog.recordEvent("releaseReservation(%d)", nBytes);
       }
     }
-
   }
 
   @Override
@@ -573,7 +578,6 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
     }
   }
 
-
   /**
    * Verifies the accounting state of the allocator. Only works for DEBUG.
    *
@@ -599,12 +603,12 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
    *           when any problems are found
    */
   private void verifyAllocator(final IdentityHashMap<UnsafeDirectLittleEndian, BaseAllocator> buffersSeen) {
-    synchronized (DEBUG_LOCK) {
+    // The remaining tests can only be performed if we're in debug mode.
+    if (!DEBUG) {
+      return;
+    }
 
-      // The remaining tests can only be performed if we're in debug mode.
-      if (!DEBUG) {
-        return;
-      }
+    synchronized (DEBUG_LOCK) {
 
       final long allocated = getAllocatedMemory();
 
@@ -757,9 +761,7 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
           reservation.historicalLog.buildHistory(sb, level + 3, true);
         }
       }
-
     }
-
   }
 
   private void dumpBuffers(final StringBuilder sb, final Set<BufferLedger> ledgerSet) {
@@ -776,7 +778,6 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
     }
   }
 
-
   public static StringBuilder indent(StringBuilder sb, int indent) {
     final char[] indentation = new char[indent * 2];
     Arrays.fill(indentation, ' ');
@@ -810,7 +811,7 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato
       // sizes provide no increase in performance.
       // Revisit from time to time.
 
-      ioBuffer = new byte[32*1024];
+      ioBuffer = new byte[IO_BUFFER_SIZE];
     }
     return ioBuffer;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/logical/src/main/java/org/apache/drill/common/expression/FieldReference.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/FieldReference.java b/logical/src/main/java/org/apache/drill/common/expression/FieldReference.java
index 640984e..ba3bf7a 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/FieldReference.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/FieldReference.java
@@ -38,7 +38,7 @@ import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 @JsonSerialize(using = Se.class)
 @JsonDeserialize(using = De.class)
 public class FieldReference extends SchemaPath {
-  MajorType overrideType;
+  private MajorType overrideType;
 
   public FieldReference(SchemaPath sp) {
     super(sp);
@@ -49,10 +49,8 @@ public class FieldReference extends SchemaPath {
     if (getRootSegment().getChild() != null) {
       throw new UnsupportedOperationException("Field references must be singular names.");
     }
-
   }
 
-
   private void checkSimpleString(CharSequence value) {
     if (value.toString().contains(".")) {
       throw new UnsupportedOperationException(
@@ -81,7 +79,6 @@ public class FieldReference extends SchemaPath {
     return new FieldReference(safeString, ExpressionPosition.UNKNOWN, false);
   }
 
-
   public FieldReference(CharSequence value, ExpressionPosition pos) {
     this(value, pos, true);
   }
@@ -92,7 +89,6 @@ public class FieldReference extends SchemaPath {
       checkData();
       checkSimpleString(value);
     }
-
   }
 
   public FieldReference(String value, ExpressionPosition pos, MajorType dataType) {
@@ -123,7 +119,6 @@ public class FieldReference extends SchemaPath {
       ref = ref.replace("`", "");
       return new FieldReference(ref, ExpressionPosition.UNKNOWN, false);
     }
-
   }
 
   @SuppressWarnings("serial")
@@ -138,7 +133,5 @@ public class FieldReference extends SchemaPath {
         JsonGenerationException {
       jgen.writeString('`' + value.getRootSegment().getNameSegment().getPath() + '`');
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index f5fc687..026fb09 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
 import org.antlr.runtime.RecognitionException;
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.PathSegment.ArraySegment;
 import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.parser.ExprLexer;
@@ -58,7 +57,6 @@ public class SchemaPath extends LogicalExpressionBase {
     return new SchemaPath(s);
   }
 
-  @SuppressWarnings("unused")
   public PathSegment getLastSegment() {
     PathSegment s= rootSegment;
     while (s.getChild() != null) {
@@ -157,7 +155,6 @@ public class SchemaPath extends LogicalExpressionBase {
     return new SchemaPath(newRoot);
   }
 
-  @SuppressWarnings("unused")
   public SchemaPath getUnindexedArrayChild() {
     NameSegment newRoot = rootSegment.cloneWithNewChild(new ArraySegment(null));
     return new SchemaPath(newRoot);

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/logical/src/main/java/org/apache/drill/common/logical/data/Order.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/Order.java b/logical/src/main/java/org/apache/drill/common/logical/data/Order.java
index 1bf587d..5cd3f84 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/data/Order.java
+++ b/logical/src/main/java/org/apache/drill/common/logical/data/Order.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,22 +19,20 @@ package org.apache.drill.common.logical.data;
 
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.util.PartiallyOrderedSet;
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelFieldCollation.Direction;
-import org.apache.calcite.rel.RelFieldCollation.NullDirection;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 
@@ -58,15 +56,15 @@ public class Order extends SingleInputOperator {
     return within;
   }
 
-    @Override
-    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
-        return logicalVisitor.visitOrder(this, value);
-    }
+  @Override
+  public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+      return logicalVisitor.visitOrder(this, value);
+  }
 
-    @Override
-    public Iterator<LogicalOperator> iterator() {
-        return Iterators.singletonIterator(getInput());
-    }
+  @Override
+  public Iterator<LogicalOperator> iterator() {
+      return Iterators.singletonIterator(getInput());
+  }
 
 
   /**
@@ -74,22 +72,33 @@ public class Order extends SingleInputOperator {
    */
   public static class Ordering {
 
+    public static final String ORDER_ASC = "ASC";
+    public static final String ORDER_DESC = "DESC";
+    public static final String ORDER_ASCENDING = "ASCENDING";
+    public static final String ORDER_DESCENDING = "DESCENDING";
+
+    public static final String NULLS_FIRST = "FIRST";
+    public static final String NULLS_LAST = "LAST";
+    public static final String NULLS_UNSPECIFIED = "UNSPECIFIED";
+
     private final LogicalExpression expr;
     /** Net &lt;ordering specification>. */
     private final Direction direction;
     /** Net &lt;null ordering> */
     private final NullDirection nullOrdering;
     /** The values in the plans for ordering specification are ASC, DESC, not the
-     * full words featured in the calcite {@link Direction} Enum, need to map between them. */
+     * full words featured in the Calcite {@link Direction} Enum, need to map between them. */
     private static ImmutableMap<String, Direction> DRILL_TO_CALCITE_DIR_MAPPING =
         ImmutableMap.<String, Direction>builder()
-        .put("ASC", Direction.ASCENDING)
-        .put("DESC", Direction.DESCENDING).build();
+        .put(ORDER_ASC, Direction.ASCENDING)
+        .put(ORDER_DESC, Direction.DESCENDING)
+        .put(ORDER_ASCENDING, Direction.ASCENDING)
+        .put(ORDER_DESCENDING, Direction.DESCENDING).build();
     private static ImmutableMap<String, NullDirection> DRILL_TO_CALCITE_NULL_DIR_MAPPING =
         ImmutableMap.<String, NullDirection>builder()
-            .put("FIRST", NullDirection.FIRST)
-            .put("LAST", NullDirection.LAST)
-            .put("UNSPECIFIED", NullDirection.UNSPECIFIED).build();
+            .put(NULLS_FIRST, NullDirection.FIRST)
+            .put(NULLS_LAST, NullDirection.LAST)
+            .put(NULLS_UNSPECIFIED, NullDirection.UNSPECIFIED).build();
 
     /**
      * Constructs a sort specification.
@@ -123,8 +132,12 @@ public class Order extends SingleInputOperator {
       this(direction, e, NullDirection.FIRST);
     }
 
-    private static Direction getOrderingSpecFromString( String strDirection ) {
-      Direction dir = DRILL_TO_CALCITE_DIR_MAPPING.get(strDirection);
+    @VisibleForTesting
+    public static Direction getOrderingSpecFromString(String strDirection) {
+      Direction dir = null;
+      if (strDirection != null) {
+        dir = DRILL_TO_CALCITE_DIR_MAPPING.get(strDirection.toUpperCase());
+      }
       if (dir != null || strDirection == null) {
         return filterDrillSupportedDirections(dir);
       } else {
@@ -134,16 +147,20 @@ public class Order extends SingleInputOperator {
       }
     }
 
-    private static NullDirection getNullOrderingFromString( String strNullOrdering ) {
-      NullDirection nullDir = DRILL_TO_CALCITE_NULL_DIR_MAPPING.get(strNullOrdering);
+    @VisibleForTesting
+    public static NullDirection getNullOrderingFromString( String strNullOrdering ) {
+      NullDirection nullDir = null;
+      if (strNullOrdering != null) {
+        nullDir = DRILL_TO_CALCITE_NULL_DIR_MAPPING.get(strNullOrdering.toUpperCase());
+      }
       if (nullDir != null || strNullOrdering == null) {
         return filterDrillSupportedNullDirections(nullDir);
       } else {
         throw new DrillRuntimeException(
             "Internal error:  Unknown <null ordering> string (not "
-                + "\"" + NullDirection.FIRST.name() + "\", "
-                + "\"" + NullDirection.LAST.name() + "\", or "
-                + "\"" + NullDirection.UNSPECIFIED.name() + "\" or null): "
+                + "\"" + NULLS_FIRST + "\", "
+                + "\"" + NULLS_LAST + "\", or "
+                + "\"" + NULLS_UNSPECIFIED + "\" or null): "
                 + "\"" + strNullOrdering + "\"" );
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3158b42..09ffbfa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -447,7 +447,8 @@
             <systemPropertyVariables>
               <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
             </systemPropertyVariables>
-          </configuration>
+			<excludedGroups>org.apache.drill.test.SecondaryTest</excludedGroups>
+		  </configuration>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>


[4/5] drill git commit: DRILL-5325: Unit tests for the managed sort

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
index 4d5f290..1dbddee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
@@ -17,47 +17,27 @@
  */
 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.ops.OperExecContext;
+import org.apache.drill.exec.ops.OperExecContextImpl;
 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.physical.impl.xsort.managed.BatchGroup.SpilledRun;
-
-import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
 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.
@@ -175,191 +155,33 @@ import com.google.common.collect.Lists;
  */
 
 public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
   protected static final ControlsInjector injector = ControlsInjectorFactory.getInjector(ExternalSortBatch.class);
 
-  /**
-   * Smallest allowed output batch size. The smallest output batch
-   * created even under constrained memory conditions.
-   */
-  private static final int MIN_MERGED_BATCH_SIZE = 256 * 1024;
-
-  /**
-   * In the bizarre case where the user gave us an unrealistically low
-   * spill file size, set a floor at some bare minimum size. (Note that,
-   * at this size, big queries will create a huge number of files, which
-   * is why the configuration default is one the order of hundreds of MB.)
-   */
-
-  private static final long MIN_SPILL_FILE_SIZE = 1 * 1024 * 1024;
-
   public static final String INTERRUPTION_AFTER_SORT = "after-sort";
   public static final String INTERRUPTION_AFTER_SETUP = "after-setup";
   public static final String INTERRUPTION_WHILE_SPILLING = "spilling";
   public static final String INTERRUPTION_WHILE_MERGING = "merging";
-  public static final long DEFAULT_SPILL_BATCH_SIZE = 8L * 1024 * 1024;
-  public static final long MIN_SPILL_BATCH_SIZE = 256 * 1024;
 
   private final RecordBatch incoming;
 
   /**
-   * Memory allocator for this operator itself. Incoming batches are
-   * transferred into this allocator. Intermediate batches used during
-   * merge also reside here.
-   */
-
-  private final BufferAllocator allocator;
-
-  /**
    * Schema of batches that this operator produces.
    */
 
   private BatchSchema schema;
 
-  /**
-   * Incoming batches buffered in memory prior to spilling
-   * or an in-memory merge.
-   */
-
-  private LinkedList<BatchGroup.InputBatch> bufferedBatches = Lists.newLinkedList();
-  private LinkedList<BatchGroup.SpilledRun> spilledRuns = Lists.newLinkedList();
-  private SelectionVector4 sv4;
-
-  /**
-   * The number of records to add to each output batch sent to the
-   * downstream operator or spilled to disk.
-   */
-
-  private int mergeBatchRowCount;
-  private int peakNumBatches = -1;
-
-  /**
-   * Maximum memory this operator may use. Usually comes from the
-   * operator definition, but may be overridden by a configuration
-   * parameter for unit testing.
-   */
-
-  private long memoryLimit;
+//  private SelectionVector4 sv4;
 
   /**
    * Iterates over the final, sorted results.
    */
 
   private SortResults resultsIterator;
-
-  /**
-   * Manages the set of spill directories and files.
-   */
-
-  private final SpillSet spillSet;
-
-  /**
-   * Manages the copier used to merge a collection of batches into
-   * a new set of batches.
-   */
-
-  private final CopierHolder copierHolder;
-
   private enum SortState { START, LOAD, DELIVER, DONE }
   private SortState sortState = SortState.START;
-  private int inputRecordCount = 0;
-  private int inputBatchCount = 0; // total number of batches received so far
-  private final OperatorCodeGenerator opCodeGen;
-
-  /**
-   * Estimated size of the records for this query, updated on each
-   * new batch received from upstream.
-   */
-
-  private int estimatedRowWidth;
-
-  /**
-   * Size of the merge batches that this operator produces. Generally
-   * the same as the merge batch size, unless low memory forces a smaller
-   * value.
-   */
-
-  private long targetMergeBatchSize;
-
-  /**
-   * Estimate of the input batch size based on the largest batch seen
-   * thus far.
-   */
-  private long estimatedInputBatchSize;
-
-  /**
-   * Maximum number of spilled runs that can be merged in a single pass.
-   */
-
-  private int mergeLimit;
-
-  /**
-   * Target size of the first-generation spill files.
-   */
-  private long spillFileSize;
-
-  /**
-   * Tracks the minimum amount of remaining memory for use
-   * in populating an operator metric.
-   */
-
-  private long minimumBufferSpace;
-
-  /**
-   * Maximum memory level before spilling occurs. That is, we can buffer input
-   * batches in memory until we reach the level given by the buffer memory pool.
-   */
-
-  private long bufferMemoryPool;
 
-  /**
-   * Maximum memory that can hold batches during the merge
-   * phase.
-   */
-
-  private long mergeMemoryPool;
-
-  /**
-   * The target size for merge batches sent downstream.
-   */
-
-  private long preferredMergeBatchSize;
-
-  /**
-   * Sum of the total number of bytes read from upstream.
-   * This is the raw memory bytes, not actual data bytes.
-   */
-
-  private long totalInputBytes;
-
-  /**
-   * The configured size for each spill batch.
-   */
-  private Long preferredSpillBatchSize;
-
-  /**
-   * Tracks the maximum density of input batches. Density is
-   * the amount of actual data / amount of memory consumed.
-   * Low density batches indicate an EOF or something wrong in
-   * an upstream operator because a low-density batch wastes
-   * memory.
-   */
-
-  private int maxDensity;
-  private int lastDensity = -1;
-
-  /**
-   * Estimated number of rows that fit into a single spill batch.
-   */
-
-  private int spillBatchRowCount;
-
-  /**
-   * The estimated actual spill batch size which depends on the
-   * details of the data rows for any particular query.
-   */
-
-  private int targetSpillBatchSize;
+  private SortImpl sortImpl;
 
   // WARNING: The enum here is used within this class. But, the members of
   // this enum MUST match those in the (unmanaged) ExternalSortBatch since
@@ -367,7 +189,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
   public enum Metric implements MetricDef {
     SPILL_COUNT,            // number of times operator spilled to disk
-    RETIRED1,               // Was: peak value for totalSizeInMemory
+    NOT_USED,               // Was: peak value for totalSizeInMemory
                             // But operator already provides this value
     PEAK_BATCHES_IN_MEMORY, // maximum number of batches kept in memory
     MERGE_COUNT,            // Number of second+ generation merges
@@ -382,126 +204,33 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     }
   }
 
-  /**
-   * Iterates over the final sorted results. Implemented differently
-   * depending on whether the results are in-memory or spilled to
-   * disk.
-   */
-
-  public interface SortResults {
-    boolean next();
-    void close();
-    int getBatchCount();
-    int getRecordCount();
-  }
-
   public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, RecordBatch incoming) {
     super(popConfig, context, true);
     this.incoming = incoming;
-    allocator = oContext.getAllocator();
-    opCodeGen = new OperatorCodeGenerator(context, popConfig);
-
-    spillSet = new SpillSet(context, popConfig, UserBitShared.CoreOperatorType.EXTERNAL_SORT);
-    copierHolder = new CopierHolder(context, allocator, opCodeGen);
-    configure(context.getConfig());
-  }
-
-  private void configure(DrillConfig config) {
-
-    // The maximum memory this operator can use as set by the
-    // operator definition (propagated to the allocator.)
-
-    memoryLimit = allocator.getLimit();
-
-    // Optional configured memory limit, typically used only for testing.
-
-    long configLimit = config.getBytes(ExecConstants.EXTERNAL_SORT_MAX_MEMORY);
-    if (configLimit > 0) {
-      memoryLimit = Math.min(memoryLimit, configLimit);
-    }
-
-    // Optional limit on the number of spilled runs to merge in a single
-    // pass. Limits the number of open file handles. Must allow at least
-    // two batches to merge to make progress.
-
-    mergeLimit = getConfigLimit(config, ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, Integer.MAX_VALUE, 2);
-
-    // Limits the size of first-generation spill files.
-
-    spillFileSize = config.getBytes(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE);
-
-    // Ensure the size is reasonable.
-
-    spillFileSize = Math.max(spillFileSize, MIN_SPILL_FILE_SIZE);
 
-    // The spill batch size. This is a critical setting for performance.
-    // Set too large and the ratio between memory and input data sizes becomes
-    // small. Set too small and disk seek times dominate performance.
+    SortConfig sortConfig = new SortConfig(context.getConfig());
+    SpillSet spillSet = new SpillSet(context.getConfig(), context.getHandle(),
+                                     popConfig);
+    OperExecContext opContext = new OperExecContextImpl(context, oContext, popConfig, injector);
+    PriorityQueueCopierWrapper copierHolder = new PriorityQueueCopierWrapper(opContext);
+    SpilledRuns spilledRuns = new SpilledRuns(opContext, spillSet, copierHolder);
+    sortImpl = new SortImpl(opContext, sortConfig, spilledRuns, container);
 
-    preferredSpillBatchSize = config.getBytes(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE);
+    // The upstream operator checks on record count before we have
+    // results. Create an empty result set temporarily to handle
+    // these calls.
 
-    // In low memory, use no more than 1/4 of memory for each spill batch. Ensures we
-    // can merge.
-
-    preferredSpillBatchSize = Math.min(preferredSpillBatchSize, memoryLimit / 4);
-
-    // But, the spill batch should be above some minimum size to prevent complete
-    // thrashing.
-
-    preferredSpillBatchSize = Math.max(preferredSpillBatchSize, MIN_SPILL_BATCH_SIZE);
-
-    // Set the target output batch size. Use the maximum size, but only if
-    // this represents less than 10% of available memory. Otherwise, use 10%
-    // of memory, but no smaller than the minimum size. In any event, an
-    // output batch can contain no fewer than a single record.
-
-    preferredMergeBatchSize = config.getBytes(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE);
-    long maxAllowance = (long) (memoryLimit - 2 * preferredSpillBatchSize);
-    preferredMergeBatchSize = Math.min(maxAllowance, preferredMergeBatchSize);
-    preferredMergeBatchSize = Math.max(preferredMergeBatchSize, MIN_MERGED_BATCH_SIZE);
-
-    logger.debug("Config: memory limit = {}, " +
-                 "spill file size = {}, spill batch size = {}, merge limit = {}, merge batch size = {}",
-                  memoryLimit, spillFileSize, preferredSpillBatchSize, mergeLimit,
-                  preferredMergeBatchSize);
-  }
-
-  private int getConfigLimit(DrillConfig config, String paramName, int valueIfZero, int minValue) {
-    int limit = config.getInt(paramName);
-    if (limit > 0) {
-      limit = Math.max(limit, minValue);
-    } else {
-      limit = valueIfZero;
-    }
-    return limit;
+    resultsIterator = new SortImpl.EmptyResults(container);
   }
 
   @Override
   public int getRecordCount() {
-    if (sv4 != null) {
-      return sv4.getCount();
-    }
-    return container.getRecordCount();
+    return resultsIterator.getRecordCount();
   }
 
   @Override
   public SelectionVector4 getSelectionVector4() {
-    return sv4;
-  }
-
-  private void closeBatchGroups(Collection<? extends BatchGroup> groups) {
-    for (BatchGroup group: groups) {
-      try {
-        group.close();
-      } catch (Exception e) {
-        // collect all failure and make sure to cleanup all remaining batches
-        // Originally we would have thrown a RuntimeException that would propagate to FragmentExecutor.closeOutResources()
-        // where it would have been passed to context.fail()
-        // passing the exception directly to context.fail(e) will let the cleanup process continue instead of stopping
-        // right away, this will also make sure we collect any additional exception we may get while cleaning up
-        context.fail(e);
-      }
-    }
+    return resultsIterator.getSv4();
   }
 
   /**
@@ -588,59 +317,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
   }
 
   /**
-   * Load and process a single batch, handling schema changes. In general, the
-   * external sort accepts only one schema.
-   *
-   * @return return code depending on the amount of data read from upstream
-   */
-
-  private IterOutcome loadBatch() {
-
-    // If this is the very first batch, then AbstractRecordBatch
-    // already loaded it for us in buildSchema().
-
-    IterOutcome upstream;
-    if (sortState == SortState.START) {
-      sortState = SortState.LOAD;
-      upstream = IterOutcome.OK_NEW_SCHEMA;
-    } else {
-      upstream = next(incoming);
-    }
-    switch (upstream) {
-    case NONE:
-    case STOP:
-      return upstream;
-    case OK_NEW_SCHEMA:
-    case OK:
-      setupSchema(upstream);
-
-      // Add the batch to the in-memory generation, spilling if
-      // needed.
-
-      processBatch();
-      break;
-    case OUT_OF_MEMORY:
-
-      // Note: it is highly doubtful that this code actually works. It
-      // requires that the upstream batches got to a safe place to run
-      // out of memory and that no work as in-flight and thus abandoned.
-      // Consider removing this case once resource management is in place.
-
-      logger.error("received OUT_OF_MEMORY, trying to spill");
-      if (bufferedBatches.size() > 2) {
-        spillFromMemory();
-      } else {
-        logger.error("not enough batches to spill, sending OUT_OF_MEMORY downstream");
-        return IterOutcome.OUT_OF_MEMORY;
-      }
-      break;
-    default:
-      throw new IllegalStateException("Unexpected iter outcome: " + upstream);
-    }
-    return IterOutcome.OK;
-  }
-
-  /**
    * Load the results and sort them. May bail out early if an exceptional
    * condition is passed up from the input batch.
    *
@@ -674,50 +350,74 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
     // Anything to actually sort?
 
-    if (inputRecordCount == 0) {
+    resultsIterator = sortImpl.startMerge();
+    if (! resultsIterator.next()) {
       sortState = SortState.DONE;
       return IterOutcome.NONE;
     }
-    logger.debug("Completed load phase: read {} batches, spilled {} times, total input bytes: {}",
-                 inputBatchCount, spilledRuns.size(), totalInputBytes);
 
-    // Do the merge of the loaded batches. The merge can be done entirely in memory if
-    // the results fit; else we have to do a disk-based merge of
-    // pre-sorted spilled batches.
+    // sort may have prematurely exited due to should continue returning false.
 
-    if (canUseMemoryMerge()) {
-      return sortInMemory();
-    } else {
-      return mergeSpilledRuns();
+    if (! context.shouldContinue()) {
+      sortState = SortState.DONE;
+      return IterOutcome.STOP;
     }
+
+    sortState = SortState.DELIVER;
+    return IterOutcome.OK_NEW_SCHEMA;
   }
 
   /**
-   * All data has been read from the upstream batch. Determine if we
-   * can use a fast in-memory sort, or must use a merge (which typically,
-   * but not always, involves spilled batches.)
+   * Load and process a single batch, handling schema changes. In general, the
+   * external sort accepts only one schema.
    *
-   * @return whether sufficient resources exist to do an in-memory sort
-   * if all batches are still in memory
+   * @return return code depending on the amount of data read from upstream
    */
 
-  private boolean canUseMemoryMerge() {
-    if (spillSet.hasSpilled()) { return false; }
+  private IterOutcome loadBatch() {
 
-    // Do we have enough memory for MSorter (the in-memory sorter)?
+    // If this is the very first batch, then AbstractRecordBatch
+    // already loaded it for us in buildSchema().
 
-    long allocMem = allocator.getAllocatedMemory();
-    long availableMem = memoryLimit - allocMem;
-    long neededForInMemorySort = MSortTemplate.memoryNeeded(inputRecordCount);
-    if (availableMem < neededForInMemorySort) { return false; }
+    IterOutcome upstream;
+    if (sortState == SortState.START) {
+      sortState = SortState.LOAD;
+      upstream = IterOutcome.OK_NEW_SCHEMA;
+    } else {
+      upstream = next(incoming);
+    }
+    switch (upstream) {
+    case NONE:
+    case STOP:
+      return upstream;
+    case OK_NEW_SCHEMA:
+      setupSchema();
+      // Fall through
 
-    // Make sure we don't exceed the maximum number of batches SV4 can address.
+    case OK:
 
-    if (bufferedBatches.size() > Character.MAX_VALUE) { return false; }
+      // Add the batch to the in-memory generation, spilling if
+      // needed.
 
-    // We can do an in-memory merge.
+      sortImpl.addBatch(incoming);
+      break;
+    case OUT_OF_MEMORY:
+
+      // Note: it is highly doubtful that this code actually works. It
+      // requires that the upstream batches got to a safe place to run
+      // out of memory and that no work was in-flight and thus abandoned.
+      // Consider removing this case once resource management is in place.
 
-    return true;
+      logger.error("received OUT_OF_MEMORY, trying to spill");
+      if (! sortImpl.forceSpill()) {
+        throw UserException.memoryError("Received OUT_OF_MEMORY, but enough batches to spill")
+          .build(logger);
+      }
+      break;
+    default:
+      throw new IllegalStateException("Unexpected iter outcome: " + upstream);
+    }
+    return IterOutcome.OK;
   }
 
   /**
@@ -727,749 +427,24 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
    * @param upstream the status code from upstream: either OK or OK_NEW_SCHEMA
    */
 
-  private void setupSchema(IterOutcome upstream)  {
+  private void setupSchema()  {
 
     // First batch: we won't have a schema.
 
     if (schema == null) {
       schema = incoming.getSchema();
-
-    // Subsequent batches, nothing to do if same schema.
-
-    } else if (upstream == IterOutcome.OK) {
-      return;
-
-    // Only change in the case that the schema truly changes. Artificial schema changes are ignored.
-
     } else if (incoming.getSchema().equals(schema)) {
-      return;
+      // Nothing to do.  Artificial schema changes are ignored.
     } else if (unionTypeEnabled) {
-        schema = SchemaUtil.mergeSchemas(schema, incoming.getSchema());
-
-        // New schema: must generate a new sorter and copier.
-
-        opCodeGen.setSchema(schema);
+      schema = SchemaUtil.mergeSchemas(schema, incoming.getSchema());
     } else {
       throw UserException.unsupportedError()
             .message("Schema changes not supported in External Sort. Please enable Union type.")
+            .addContext("Previous schema", schema.toString())
+            .addContext("Incoming schema", incoming.getSchema().toString())
             .build(logger);
     }
-
-    // Coerce all existing batches to the new schema.
-
-    for (BatchGroup b : bufferedBatches) {
-      b.setSchema(schema);
-    }
-    for (BatchGroup b : spilledRuns) {
-      b.setSchema(schema);
-    }
-  }
-
-  /**
-   * Convert an incoming batch into the agree-upon format. (Also seems to
-   * make a persistent shallow copy of the batch saved until we are ready
-   * to sort or spill.)
-   *
-   * @return the converted batch, or null if the incoming batch is empty
-   */
-
-  @SuppressWarnings("resource")
-  private VectorContainer convertBatch() {
-
-    // Must accept the batch even if no records. Then clear
-    // the vectors to release memory since we won't do any
-    // further processing with the empty batch.
-
-    VectorContainer convertedBatch = SchemaUtil.coerceContainer(incoming, schema, oContext);
-    if (incoming.getRecordCount() == 0) {
-      for (VectorWrapper<?> w : convertedBatch) {
-        w.clear();
-      }
-      SelectionVector2 sv2 = incoming.getSelectionVector2();
-      if (sv2 != null) {
-        sv2.clear();
-      }
-      return null;
-    }
-    return convertedBatch;
-  }
-
-  private SelectionVector2 makeSelectionVector() {
-    if (incoming.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE) {
-      return incoming.getSelectionVector2().clone();
-    } else {
-      return newSV2();
-    }
-  }
-
-  /**
-   * Process the converted incoming batch by adding it to the in-memory store
-   * of data, or spilling data to disk when necessary.
-   */
-
-  @SuppressWarnings("resource")
-  private void processBatch() {
-
-    // Skip empty batches (such as the first one.)
-
-    if (incoming.getRecordCount() == 0) {
-      return;
-    }
-
-    // Determine actual sizes of the incoming batch before taking
-    // ownership. Allows us to figure out if we need to spill first,
-    // to avoid overflowing memory simply due to ownership transfer.
-
-    RecordBatchSizer sizer = analyzeIncomingBatch();
-
-    // The heart of the external sort operator: spill to disk when
-    // the in-memory generation exceeds the allowed memory limit.
-    // Preemptively spill BEFORE accepting the new batch into our memory
-    // pool. The allocator will throw an OOM exception if we accept the
-    // batch when we are near the limit - despite the fact that the batch
-    // is already in memory and no new memory is allocated during the transfer.
-
-    if ( isSpillNeeded(sizer.actualSize())) {
-      spillFromMemory();
-    }
-
-    // Sanity check. We should now be below the buffer memory maximum.
-
-    long startMem = allocator.getAllocatedMemory();
-    if (startMem > bufferMemoryPool) {
-      logger.error( "ERROR: Failed to spill above buffer limit. Buffer pool = {}, memory = {}",
-          bufferMemoryPool, startMem);
-    }
-
-    // Convert the incoming batch to the agreed-upon schema.
-    // No converted batch means we got an empty input batch.
-    // Converting the batch transfers memory ownership to our
-    // allocator. This gives a round-about way to learn the batch
-    // size: check the before and after memory levels, then use
-    // the difference as the batch size, in bytes.
-
-    VectorContainer convertedBatch = convertBatch();
-    if (convertedBatch == null) {
-      return;
-    }
-
-    SelectionVector2 sv2;
-    try {
-      sv2 = makeSelectionVector();
-    } catch (Exception e) {
-      convertedBatch.clear();
-      throw e;
-    }
-
-    // Compute batch size, including allocation of an sv2.
-
-    long endMem = allocator.getAllocatedMemory();
-    long batchSize = endMem - startMem;
-    int count = sv2.getCount();
-    inputRecordCount += count;
-    inputBatchCount++;
-    totalInputBytes += sizer.actualSize();
-
-    // Update the minimum buffer space metric.
-
-    if (minimumBufferSpace == 0) {
-      minimumBufferSpace = endMem;
-    } else {
-      minimumBufferSpace = Math.min(minimumBufferSpace, endMem);
-    }
-    stats.setLongStat(Metric.MIN_BUFFER, minimumBufferSpace);
-
-    // Update the size based on the actual record count, not
-    // the effective count as given by the selection vector
-    // (which may exclude some records due to filtering.)
-
-    updateMemoryEstimates(batchSize, sizer);
-
-    // Sort the incoming batch using either the original selection vector,
-    // or a new one created here.
-
-    SingleBatchSorter sorter;
-    sorter = opCodeGen.getSorter(convertedBatch);
-    try {
-      sorter.setup(context, sv2, convertedBatch);
-    } catch (SchemaChangeException e) {
-      convertedBatch.clear();
-      throw UserException.unsupportedError(e)
-            .message("Unexpected schema change.")
-            .build(logger);
-    }
-    try {
-      sorter.sort(sv2);
-    } catch (SchemaChangeException e) {
-      convertedBatch.clear();
-      throw UserException.unsupportedError(e)
-                .message("Unexpected schema change.")
-                .build(logger);
-    }
-    RecordBatchData rbd = new RecordBatchData(convertedBatch, allocator);
-    try {
-      rbd.setSv2(sv2);
-      bufferedBatches.add(new BatchGroup.InputBatch(rbd.getContainer(), rbd.getSv2(), oContext, sizer.netSize()));
-      if (peakNumBatches < bufferedBatches.size()) {
-        peakNumBatches = bufferedBatches.size();
-        stats.setLongStat(Metric.PEAK_BATCHES_IN_MEMORY, peakNumBatches);
-      }
-
-    } catch (Throwable t) {
-      rbd.clear();
-      throw t;
-    }
-  }
-
-  /**
-   * Scan the vectors in the incoming batch to determine batch size and if
-   * any oversize columns exist. (Oversize columns cause memory fragmentation.)
-   *
-   * @return an analysis of the incoming batch
-   */
-
-  private RecordBatchSizer analyzeIncomingBatch() {
-    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
-    sizer.applySv2();
-    if (inputBatchCount == 0) {
-      logger.debug("{}", sizer.toString());
-    }
-    return sizer;
-  }
-
-  /**
-   * Update the data-driven memory use numbers including:
-   * <ul>
-   * <li>The average size of incoming records.</li>
-   * <li>The estimated spill and output batch size.</li>
-   * <li>The estimated number of average-size records per
-   * spill and output batch.</li>
-   * <li>The amount of memory set aside to hold the incoming
-   * batches before spilling starts.</li>
-   * </ul>
-   *
-   * @param actualBatchSize the overall size of the current batch received from
-   * upstream
-   * @param actualRecordCount the number of actual (not filtered) records in
-   * that upstream batch
-   */
-
-  private void updateMemoryEstimates(long memoryDelta, RecordBatchSizer sizer) {
-    long actualBatchSize = sizer.actualSize();
-    int actualRecordCount = sizer.rowCount();
-
-    if (actualBatchSize != memoryDelta) {
-      logger.debug("Memory delta: {}, actual batch size: {}, Diff: {}",
-                   memoryDelta, actualBatchSize, memoryDelta - actualBatchSize);
-    }
-
-    // The record count should never be zero, but better safe than sorry...
-
-    if (actualRecordCount == 0) {
-      return; }
-
-    // If the vector is less than 75% full, just ignore it, except in the
-    // unfortunate case where it is the first batch. Low-density batches generally
-    // occur only at the end of a file or at the end of a DFS block. In such a
-    // case, we will continue to rely on estimates created on previous, high-
-    // density batches.
-    // We actually track the max density seen, and compare to 75% of that since
-    // Parquet produces very low density record batches.
-
-    if (sizer.avgDensity() < maxDensity * 3 / 4 && sizer.avgDensity() != lastDensity) {
-      logger.trace("Saw low density batch. Density: {}", sizer.avgDensity());
-      lastDensity = sizer.avgDensity();
-      return;
-    }
-    maxDensity = Math.max(maxDensity, sizer.avgDensity());
-
-    // We know the batch size and number of records. Use that to estimate
-    // the average record size. Since a typical batch has many records,
-    // the average size is a fairly good estimator. Note that the batch
-    // size includes not just the actual vector data, but any unused space
-    // resulting from power-of-two allocation. This means that we don't
-    // have to do size adjustments for input batches as we will do below
-    // when estimating the size of other objects.
-
-    int batchRowWidth = sizer.netRowWidth();
-
-    // Record sizes may vary across batches. To be conservative, use
-    // the largest size observed from incoming batches.
-
-    int origRowEstimate = estimatedRowWidth;
-    estimatedRowWidth = Math.max(estimatedRowWidth, batchRowWidth);
-
-    // Maintain an estimate of the incoming batch size: the largest
-    // batch yet seen. Used to reserve memory for the next incoming
-    // batch. Because we are using the actual observed batch size,
-    // the size already includes overhead due to power-of-two rounding.
-
-    long origInputBatchSize = estimatedInputBatchSize;
-    estimatedInputBatchSize = Math.max(estimatedInputBatchSize, actualBatchSize);
-
-    // The row width may end up as zero if all fields are nulls or some
-    // other unusual situation. In this case, assume a width of 10 just
-    // to avoid lots of special case code.
-
-    if (estimatedRowWidth == 0) {
-      estimatedRowWidth = 10;
-    }
-
-    // Go no further if nothing changed.
-
-    if (estimatedRowWidth == origRowEstimate && estimatedInputBatchSize == origInputBatchSize) {
-      return; }
-
-    // Estimate the total size of each incoming batch plus sv2. Note that, due
-    // to power-of-two rounding, the allocated sv2 size might be twice the data size.
-
-    long estimatedInputSize = estimatedInputBatchSize + 4 * actualRecordCount;
-
-    // Determine the number of records to spill per spill batch. The goal is to
-    // spill batches of either 64K records, or as many records as fit into the
-    // amount of memory dedicated to each spill batch, whichever is less.
-
-    spillBatchRowCount = (int) Math.max(1, preferredSpillBatchSize / estimatedRowWidth / 2);
-    spillBatchRowCount = Math.min(spillBatchRowCount, Character.MAX_VALUE);
-
-    // Compute the actual spill batch size which may be larger or smaller
-    // than the preferred size depending on the row width. Double the estimated
-    // memory needs to allow for power-of-two rounding.
-
-    targetSpillBatchSize = spillBatchRowCount * estimatedRowWidth * 2;
-
-    // Determine the number of records per batch per merge step. The goal is to
-    // merge batches of either 64K records, or as many records as fit into the
-    // amount of memory dedicated to each merge batch, whichever is less.
-
-    mergeBatchRowCount = (int) Math.max(1, preferredMergeBatchSize / estimatedRowWidth / 2);
-    mergeBatchRowCount = Math.min(mergeBatchRowCount, Character.MAX_VALUE);
-    mergeBatchRowCount = Math.max(1,  mergeBatchRowCount);
-    targetMergeBatchSize = mergeBatchRowCount * estimatedRowWidth * 2;
-
-    // Determine the minimum memory needed for spilling. Spilling is done just
-    // before accepting a batch, so we must spill if we don't have room for a
-    // (worst case) input batch. To spill, we need room for the output batch created
-    // by merging the batches already in memory. Double this to allow for power-of-two
-    // memory allocations.
-
-    long spillPoint = estimatedInputBatchSize + 2 * targetSpillBatchSize;
-
-    // The merge memory pool assumes we can spill all input batches. To make
-    // progress, we must have at least two merge batches (same size as an output
-    // batch) and one output batch. Again, double to allow for power-of-two
-    // allocation and add one for a margin of error.
-
-    long minMergeMemory = 2 * targetSpillBatchSize + targetMergeBatchSize;
-
-    // If we are in a low-memory condition, then we might not have room for the
-    // default output batch size. In that case, pick a smaller size.
-
-    if (minMergeMemory > memoryLimit) {
-
-      // Figure out the minimum output batch size based on memory,
-      // must hold at least one complete row.
-
-      long mergeAllowance = memoryLimit - 2 * targetSpillBatchSize;
-      targetMergeBatchSize = Math.max(estimatedRowWidth, mergeAllowance / 2);
-      mergeBatchRowCount = (int) (targetMergeBatchSize / estimatedRowWidth / 2);
-      minMergeMemory = 2 * targetSpillBatchSize + targetMergeBatchSize;
-    }
-
-    // Determine the minimum total memory we would need to receive two input
-    // batches (the minimum needed to make progress) and the allowance for the
-    // output batch.
-
-    long minLoadMemory = spillPoint + estimatedInputSize;
-
-    // Determine how much memory can be used to hold in-memory batches of spilled
-    // runs when reading from disk.
-
-    bufferMemoryPool = memoryLimit - spillPoint;
-    mergeMemoryPool = Math.max(memoryLimit - minMergeMemory,
-                               (long) ((memoryLimit - 3 * targetMergeBatchSize) * 0.95));
-
-    // Sanity check: if we've been given too little memory to make progress,
-    // issue a warning but proceed anyway. Should only occur if something is
-    // configured terribly wrong.
-
-    long minMemoryNeeds = Math.max(minLoadMemory, minMergeMemory);
-    if (minMemoryNeeds > memoryLimit) {
-      logger.warn("Potential memory overflow! " +
-                   "Minumum needed = {} bytes, actual available = {} bytes",
-                   minMemoryNeeds, memoryLimit);
-    }
-
-    // Log the calculated values. Turn this on if things seem amiss.
-    // Message will appear only when the values change.
-
-    logger.debug("Input Batch Estimates: record size = {} bytes; input batch = {} bytes, {} records",
-                 estimatedRowWidth, estimatedInputBatchSize, actualRecordCount);
-    logger.debug("Merge batch size = {} bytes, {} records; spill file size: {} bytes",
-                 targetSpillBatchSize, spillBatchRowCount, spillFileSize);
-    logger.debug("Output batch size = {} bytes, {} records",
-                 targetMergeBatchSize, mergeBatchRowCount);
-    logger.debug("Available memory: {}, buffer memory = {}, merge memory = {}",
-                 memoryLimit, bufferMemoryPool, mergeMemoryPool);
-  }
-
-  /**
-   * Determine if spill is needed before receiving the new record batch.
-   * Spilling is driven purely by memory availability (and an optional
-   * batch limit for testing.)
-   *
-   * @return true if spilling is needed, false otherwise
-   */
-
-  private boolean isSpillNeeded(int incomingSize) {
-
-    // Can't spill if less than two batches else the merge
-    // can't make progress.
-
-    if (bufferedBatches.size() < 2) {
-      return false; }
-
-    // Must spill if we are below the spill point (the amount of memory
-    // needed to do the minimal spill.)
-
-    return allocator.getAllocatedMemory() + incomingSize >= bufferMemoryPool;
-  }
-
-  /**
-   * Perform an in-memory sort of the buffered batches. Obviously can
-   * be used only for the non-spilling case.
-   *
-   * @return DONE if no rows, OK_NEW_SCHEMA if at least one row
-   */
-
-  private IterOutcome sortInMemory() {
-    logger.debug("Starting in-memory sort. Batches = {}, Records = {}, Memory = {}",
-                 bufferedBatches.size(), inputRecordCount, allocator.getAllocatedMemory());
-
-    // Note the difference between how we handle batches here and in the spill/merge
-    // case. In the spill/merge case, this class decides on the batch size to send
-    // downstream. However, in the in-memory case, we must pass along all batches
-    // in a single SV4. Attempts to do paging will result in errors. In the memory
-    // merge case, the downstream Selection Vector Remover will split the one
-    // big SV4 into multiple smaller batches to send further downstream.
-
-    // If the sort fails or is empty, clean up here. Otherwise, cleanup is done
-    // by closing the resultsIterator after all results are returned downstream.
-
-    MergeSort memoryMerge = new MergeSort(context, allocator, opCodeGen);
-    try {
-      sv4 = memoryMerge.merge(bufferedBatches, this, container);
-      if (sv4 == null) {
-        sortState = SortState.DONE;
-        return IterOutcome.STOP;
-      } else {
-        logger.debug("Completed in-memory sort. Memory = {}",
-                     allocator.getAllocatedMemory());
-        resultsIterator = memoryMerge;
-        memoryMerge = null;
-        sortState = SortState.DELIVER;
-        return IterOutcome.OK_NEW_SCHEMA;
-      }
-    } finally {
-      if (memoryMerge != null) {
-        memoryMerge.close();
-      }
-    }
-  }
-
-  /**
-   * Perform merging of (typically spilled) batches. First consolidates batches
-   * as needed, then performs a final merge that is read one batch at a time
-   * to deliver batches to the downstream operator.
-   *
-   * @return always returns OK_NEW_SCHEMA
-   */
-
-  private IterOutcome mergeSpilledRuns() {
-    logger.debug("Starting consolidate phase. Batches = {}, Records = {}, Memory = {}, In-memory batches {}, spilled runs {}",
-                 inputBatchCount, inputRecordCount, allocator.getAllocatedMemory(),
-                 bufferedBatches.size(), spilledRuns.size());
-
-    // Consolidate batches to a number that can be merged in
-    // a single last pass.
-
-    int mergeCount = 0;
-    while (consolidateBatches()) {
-      mergeCount++;
-    }
-    stats.addLongStat(Metric.MERGE_COUNT, mergeCount);
-
-    // Merge in-memory batches and spilled runs for the final merge.
-
-    List<BatchGroup> allBatches = new LinkedList<>();
-    allBatches.addAll(bufferedBatches);
-    bufferedBatches.clear();
-    allBatches.addAll(spilledRuns);
-    spilledRuns.clear();
-
-    logger.debug("Starting merge phase. Runs = {}, Alloc. memory = {}",
-                 allBatches.size(), allocator.getAllocatedMemory());
-
-    // Do the final merge as a results iterator.
-
-    CopierHolder.BatchMerger merger = copierHolder.startFinalMerge(schema, allBatches, container, mergeBatchRowCount);
-    merger.next();
-    resultsIterator = merger;
-    sortState = SortState.DELIVER;
-    return IterOutcome.OK_NEW_SCHEMA;
-  }
-
-  private boolean consolidateBatches() {
-
-    // Determine additional memory needed to hold one batch from each
-    // spilled run.
-
-    int inMemCount = bufferedBatches.size();
-    int spilledRunsCount = spilledRuns.size();
-
-    // Can't merge more than will fit into memory at one time.
-
-    int maxMergeWidth = (int) (mergeMemoryPool / targetSpillBatchSize);
-    maxMergeWidth = Math.min(mergeLimit, maxMergeWidth);
-
-    // But, must merge at least two batches.
-
-    maxMergeWidth = Math.max(maxMergeWidth, 2);
-
-    // If we can't fit all batches in memory, must spill any in-memory
-    // batches to make room for multiple spill-merge-spill cycles.
-
-    if (inMemCount > 0) {
-      if (spilledRunsCount > maxMergeWidth) {
-        spillFromMemory();
-        return true;
-      }
-
-      // If we just plain have too many batches to merge, spill some
-      // in-memory batches to reduce the burden.
-
-      if (inMemCount + spilledRunsCount > mergeLimit) {
-        spillFromMemory();
-        return true;
-      }
-
-      // If the on-disk batches and in-memory batches need more memory than
-      // is available, spill some in-memory batches.
-
-      long allocated = allocator.getAllocatedMemory();
-      long totalNeeds = spilledRunsCount * targetSpillBatchSize + allocated;
-      if (totalNeeds > mergeMemoryPool) {
-        spillFromMemory();
-        return true;
-      }
-    }
-
-    // Merge on-disk batches if we have too many.
-
-    int mergeCount = spilledRunsCount - maxMergeWidth;
-    if (mergeCount <= 0) {
-      return false;
-    }
-
-    // Must merge at least 2 batches to make progress.
-
-    mergeCount = Math.max(2, mergeCount);
-
-    // We will merge. This will create yet another spilled
-    // run. Account for that.
-
-    mergeCount += 1;
-
-    mergeCount = Math.min(mergeCount, maxMergeWidth);
-
-    // If we are going to merge, and we have batches in memory,
-    // spill them and try again. We need to do this to ensure we
-    // have adequate memory to hold the merge batches. We are into
-    // a second-generation sort/merge so there is no point in holding
-    // onto batches in memory.
-
-    if (inMemCount > 0) {
-      spillFromMemory();
-      return true;
-    }
-
-    // Do the merge, then loop to try again in case not
-    // all the target batches spilled in one go.
-
-    logger.trace("Merging {} on-disk runs, Alloc. memory = {}",
-        mergeCount, allocator.getAllocatedMemory());
-    mergeRuns(mergeCount);
-    return true;
-  }
-
-  /**
-   * This operator has accumulated a set of sorted incoming record batches.
-   * We wish to spill some of them to disk. To do this, a "copier"
-   * merges the target batches to produce a stream of new (merged) batches
-   * which are then written to disk.
-   * <p>
-   * This method spills only half the accumulated batches
-   * minimizing unnecessary disk writes. The exact count must lie between
-   * the minimum and maximum spill counts.
-   */
-
-  private void spillFromMemory() {
-
-    // Determine the number of batches to spill to create a spill file
-    // of the desired size. The actual file size might be a bit larger
-    // or smaller than the target, which is expected.
-
-    int spillCount = 0;
-    long spillSize = 0;
-    for (InputBatch batch : bufferedBatches) {
-      long batchSize = batch.getDataSize();
-      spillSize += batchSize;
-      spillCount++;
-      if (spillSize + batchSize / 2 > spillFileSize) {
-        break; }
-    }
-
-    // Must always spill at least 2, even if this creates an over-size
-    // spill file. But, if this is a final consolidation, we may have only
-    // a single batch.
-
-    spillCount = Math.max(spillCount, 2);
-    spillCount = Math.min(spillCount, bufferedBatches.size());
-
-    // Do the actual spill.
-
-    mergeAndSpill(bufferedBatches, spillCount);
-  }
-
-  private void mergeRuns(int targetCount) {
-
-    // Determine the number of runs to merge. The count should be the
-    // target count. However, to prevent possible memory overrun, we
-    // double-check with actual spill batch size and only spill as much
-    // as fits in the merge memory pool.
-
-    int mergeCount = 0;
-    long mergeSize = 0;
-    for (SpilledRun run : spilledRuns) {
-      long batchSize = run.getBatchSize();
-      if (mergeSize + batchSize > mergeMemoryPool) {
-        break;
-      }
-      mergeSize += batchSize;
-      mergeCount++;
-      if (mergeCount == targetCount) {
-        break;
-      }
-    }
-
-    // Must always spill at least 2, even if this creates an over-size
-    // spill file. But, if this is a final consolidation, we may have only
-    // a single batch.
-
-    mergeCount = Math.max(mergeCount, 2);
-    mergeCount = Math.min(mergeCount, spilledRuns.size());
-
-    // Do the actual spill.
-
-    mergeAndSpill(spilledRuns, mergeCount);
-  }
-
-  private void mergeAndSpill(LinkedList<? extends BatchGroup> source, int count) {
-    spilledRuns.add(doMergeAndSpill(source, count));
-    logger.trace("Completed spill: memory = {}",
-                 allocator.getAllocatedMemory());
-  }
-
-  private BatchGroup.SpilledRun doMergeAndSpill(LinkedList<? extends BatchGroup> batchGroups, int spillCount) {
-    List<BatchGroup> batchesToSpill = Lists.newArrayList();
-    spillCount = Math.min(batchGroups.size(), spillCount);
-    assert spillCount > 0 : "Spill count to mergeAndSpill must not be zero";
-    for (int i = 0; i < spillCount; i++) {
-      batchesToSpill.add(batchGroups.pollFirst());
-    }
-
-    // Merge the selected set of matches and write them to the
-    // spill file. After each write, we release the memory associated
-    // with the just-written batch.
-
-    String outputFile = spillSet.getNextSpillFile(null);
-    stats.setLongStat(Metric.SPILL_COUNT, spillSet.getFileCount());
-    BatchGroup.SpilledRun newGroup = null;
-    try (AutoCloseable ignored = AutoCloseables.all(batchesToSpill);
-         CopierHolder.BatchMerger merger = copierHolder.startMerge(schema, batchesToSpill, spillBatchRowCount)) {
-      logger.trace("Spilling {} of {} batches, spill batch size = {} rows, memory = {}, write to {}",
-                   batchesToSpill.size(), bufferedBatches.size() + batchesToSpill.size(),
-                   spillBatchRowCount,
-                   allocator.getAllocatedMemory(), outputFile);
-      newGroup = new BatchGroup.SpilledRun(spillSet, outputFile, oContext);
-
-      // The copier will merge records from the buffered batches into
-      // the outputContainer up to targetRecordCount number of rows.
-      // The actual count may be less if fewer records are available.
-
-      while (merger.next()) {
-
-        // Add a new batch of records (given by merger.getOutput()) to the spill
-        // file.
-        //
-        // note that addBatch also clears the merger's output container
-
-        newGroup.addBatch(merger.getOutput());
-      }
-      injector.injectChecked(context.getExecutionControls(), INTERRUPTION_WHILE_SPILLING, IOException.class);
-      newGroup.closeOutputStream();
-      logger.trace("Spilled {} batches, {} records; memory = {} to {}",
-                   merger.getBatchCount(), merger.getRecordCount(),
-                   allocator.getAllocatedMemory(), outputFile);
-      newGroup.setBatchSize(merger.getEstBatchSize());
-      return newGroup;
-    } catch (Throwable e) {
-      // we only need to clean up newGroup if spill failed
-      try {
-        if (newGroup != null) {
-          AutoCloseables.close(e, newGroup);
-        }
-      } catch (Throwable t) { /* close() may hit the same IO issue; just ignore */ }
-
-      // Here the merger is holding onto a partially-completed batch.
-      // It will release the memory in the close() call.
-
-      try {
-        // Rethrow so we can decide how to handle the error.
-
-        throw e;
-      }
-
-      // If error is a User Exception, just use as is.
-
-      catch (UserException ue) { throw ue; }
-      catch (Throwable ex) {
-        throw UserException.resourceError(ex)
-              .message("External Sort encountered an error while spilling to disk")
-              .build(logger);
-      }
-    }
-  }
-
-  /**
-   * Allocate and initialize the selection vector used as the sort index.
-   * Assumes that memory is available for the vector since memory management
-   * ensured space is available.
-   *
-   * @return a new, populated selection vector 2
-   */
-
-  private SelectionVector2 newSV2() {
-    SelectionVector2 sv2 = new SelectionVector2(allocator);
-    if (!sv2.allocateNewSafe(incoming.getRecordCount())) {
-      throw UserException.resourceError(new OutOfMemoryException("Unable to allocate sv2 buffer"))
-            .build(logger);
-    }
-    for (int i = 0; i < incoming.getRecordCount(); i++) {
-      sv2.setIndex(i, (char) i);
-    }
-    sv2.setRecordCount(incoming.getRecordCount());
-    return sv2;
+    sortImpl.setSchema(schema);
   }
 
   @Override
@@ -1494,37 +469,8 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
   @Override
   public void close() {
-    if (spillSet.getWriteBytes() > 0) {
-      logger.debug("End of sort. Total write bytes: {}, Total read bytes: {}",
-                   spillSet.getWriteBytes(), spillSet.getWriteBytes());
-    }
-    stats.setLongStat(Metric.SPILL_MB,
-        (int) Math.round( spillSet.getWriteBytes() / 1024.0D / 1024.0 ) );
     RuntimeException ex = null;
     try {
-      if (bufferedBatches != null) {
-        closeBatchGroups(bufferedBatches);
-        bufferedBatches = null;
-      }
-    } catch (RuntimeException e) {
-      ex = e;
-    }
-    try {
-      if (spilledRuns != null) {
-        closeBatchGroups(spilledRuns);
-        spilledRuns = null;
-      }
-    } catch (RuntimeException e) {
-      ex = (ex == null) ? e : ex;
-    }
-    try {
-      if (sv4 != null) {
-        sv4.clear();
-      }
-    } catch (RuntimeException e) {
-      ex = (ex == null) ? e : ex;
-    }
-    try {
       if (resultsIterator != null) {
         resultsIterator.close();
         resultsIterator = null;
@@ -1533,17 +479,10 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       ex = (ex == null) ? e : ex;
     }
     try {
-      copierHolder.close();
-    } catch (RuntimeException e) {
-      ex = (ex == null) ? e : ex;
-    }
-    try {
-      spillSet.close();
-    } catch (RuntimeException e) {
-      ex = (ex == null) ? e : ex;
-    }
-    try {
-      opCodeGen.close();
+      if (sortImpl != null) {
+        sortImpl.close();
+        sortImpl = null;
+      }
     } catch (RuntimeException e) {
       ex = (ex == null) ? e : ex;
     }
@@ -1558,11 +497,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       ex = (ex == null) ? e : ex;
     }
     // Note: allocator is closed by the FragmentManager
-//    try {
-//      allocator.close();
-//    } catch (RuntimeException e) {
-//      ex = (ex == null) ? e : ex;
-//    }
     if (ex != null) {
       throw ex;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
index 31475d2..da41e5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSortTemplate.java
@@ -24,7 +24,7 @@ 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.ops.FragmentExecContext;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector4;
@@ -49,7 +49,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
    */
 
   private Queue<Integer> runStarts = Queues.newLinkedBlockingQueue();
-  private FragmentContext context;
+  private FragmentExecContext context;
 
   /**
    * Controls the maximum size of batches exposed to downstream
@@ -57,7 +57,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
   private int desiredRecordBatchCount;
 
   @Override
-  public void setup(final FragmentContext context, final BufferAllocator allocator, final SelectionVector4 vector4,
+  public void setup(final FragmentExecContext 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);
@@ -147,7 +147,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
   }
 
   /**
-   * Sort (really, merge) a set of pre-sorted runs to produce a combined
+   * Merge a set of pre-sorted runs to produce a combined
    * result set. Merging is done in the selection vector, record data does
    * not move.
    * <p>
@@ -157,7 +157,7 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
    */
 
   @Override
-  public void sort(final VectorContainer container) {
+  public void sort() {
     while (runStarts.size() > 1) {
       final int totalCount = this.vector4.getTotalCount();
 
@@ -223,15 +223,21 @@ public abstract class MSortTemplate implements MSorter, IndexedSortable {
 
   @Override
   public void clear() {
-    if(vector4 != null) {
+    if (vector4 != null) {
       vector4.clear();
+      vector4 = null;
     }
-
-    if(aux != null) {
+    if (aux != null) {
       aux.clear();
+      aux = null;
     }
   }
 
-  public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorContainer incoming, @Named("outgoing") RecordBatch outgoing) throws SchemaChangeException;
-  public abstract int doEval(@Named("leftIndex") int leftIndex, @Named("rightIndex") int rightIndex) throws SchemaChangeException;
+  public abstract void doSetup(@Named("context") FragmentExecContext context,
+                               @Named("incoming") VectorContainer incoming,
+                               @Named("outgoing") RecordBatch outgoing)
+                       throws SchemaChangeException;
+  public abstract int doEval(@Named("leftIndex") int leftIndex,
+                             @Named("rightIndex") int rightIndex)
+                      throws SchemaChangeException;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
index 0d04b7e..06bbdea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MSorter.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.physical.impl.xsort.managed;
 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.ops.FragmentExecContext;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 
@@ -30,8 +30,8 @@ import org.apache.drill.exec.record.selection.SelectionVector4;
  */
 
 public interface MSorter {
-  public void setup(FragmentContext context, BufferAllocator allocator, SelectionVector4 vector4, VectorContainer hyperBatch, int outputBatchSize) throws SchemaChangeException;
-  public void sort(VectorContainer container);
+  public void setup(FragmentExecContext context, BufferAllocator allocator, SelectionVector4 vector4, VectorContainer hyperBatch, int outputBatchSize) throws SchemaChangeException;
+  public void sort();
   public SelectionVector4 getSV4();
 
   public static TemplateClassDefinition<MSorter> TEMPLATE_DEFINITION = new TemplateClassDefinition<MSorter>(MSorter.class, MSortTemplate.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSort.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSort.java
deleted file mode 100644
index c3e2dbe..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSort.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * 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;
-
-/**
- * Wrapper around the "MSorter" (in memory merge sorter). As batches have
- * arrived to the sort, they have been individually sorted and buffered
- * in memory. At the completion of the sort, we detect that no batches
- * were spilled to disk. In this case, we can merge the in-memory batches
- * using an efficient memory-based approach implemented here.
- * <p>
- * Since all batches are in memory, we don't want to use the usual merge
- * algorithm as that makes a copy of the original batches (which were read
- * from a spill file) to produce an output batch. Instead, we want to use
- * the in-memory batches as-is. To do this, we use a selection vector 4
- * (SV4) as a global index into the collection of batches. The SV4 uses
- * the upper two bytes as the batch index, and the lower two as an offset
- * of a record within the batch.
- * <p>
- * The merger ("M Sorter") populates the SV4 by scanning the set of
- * in-memory batches, searching for the one with the lowest value of the
- * sort key. The batch number and offset are placed into the SV4. The process
- * continues until all records from all batches have an entry in the SV4.
- * <p>
- * The actual implementation uses an iterative merge to perform the above
- * efficiently.
- * <p>
- * A sort can only do a single merge. So, we do not attempt to share the
- * generated class; we just generate it internally and discard it at
- * completion of the merge.
- */
-
-public class MergeSort implements SortResults {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeSort.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 MergeSort(FragmentContext context, BufferAllocator allocator, OperatorCodeGenerator opCg) {
-    this.context = context;
-    this.oAllocator = allocator;
-    this.opCg = opCg;
-  }
-
-  /**
-   * Merge the set of in-memory batches to produce a single logical output in the given
-   * destination container, indexed by an SV4.
-   *
-   * @param batchGroups the complete set of in-memory batches
-   * @param batch the record batch (operator) for the sort operator
-   * @param destContainer the vector container for the sort operator
-   * @return the sv4 for this operator
-   */
-
-  public SelectionVector4 merge(LinkedList<BatchGroup.InputBatch> batchGroups, VectorAccessible batch,
-                                VectorContainer destContainer) {
-
-    // Add the buffered batches to a collection that MSorter can use.
-    // The builder takes ownership of the batches and will release them if
-    // an error occurs.
-
-    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();
-
-    // Generate the msorter.
-
-    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;
-  }
-
-  /**
-   * The SV4 provides a built-in iterator that returns a virtual set of record
-   * batches so that the downstream operator need not consume the entire set
-   * of accumulated batches in a single step.
-   */
-
-  @Override
-  public boolean next() {
-    boolean more = sv4.next();
-    if (more) { batchCount++; }
-    return more;
-  }
-
-  @Override
-  public void close() {
-    if (builder != null) {
-      builder.clear();
-      builder.close();
-    }
-    if (mSorter != null) {
-      mSorter.clear();
-    }
-  }
-
-  @Override
-  public int getBatchCount() {
-    return batchCount;
-  }
-
-  @Override
-  public int getRecordCount() {
-    return sv4.getTotalCount();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
new file mode 100644
index 0000000..3ab9af3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.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;
+
+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.MappingSet;
+import org.apache.drill.exec.exception.SchemaChangeException;
+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.OperExecContext;
+import org.apache.drill.exec.physical.config.Sort;
+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.SortImpl.SortResults;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+
+/**
+ * Wrapper around the "MSorter" (in memory merge sorter). As batches have
+ * arrived to the sort, they have been individually sorted and buffered
+ * in memory. At the completion of the sort, we detect that no batches
+ * were spilled to disk. In this case, we can merge the in-memory batches
+ * using an efficient memory-based approach implemented here.
+ * <p>
+ * Since all batches are in memory, we don't want to use the usual merge
+ * algorithm as that makes a copy of the original batches (which were read
+ * from a spill file) to produce an output batch. Instead, we want to use
+ * the in-memory batches as-is. To do this, we use a selection vector 4
+ * (SV4) as a global index into the collection of batches. The SV4 uses
+ * the upper two bytes as the batch index, and the lower two as an offset
+ * of a record within the batch.
+ * <p>
+ * The merger ("M Sorter") populates the SV4 by scanning the set of
+ * in-memory batches, searching for the one with the lowest value of the
+ * sort key. The batch number and offset are placed into the SV4. The process
+ * continues until all records from all batches have an entry in the SV4.
+ * <p>
+ * The actual implementation uses an iterative merge to perform the above
+ * efficiently.
+ * <p>
+ * A sort can only do a single merge. So, we do not attempt to share the
+ * generated class; we just generate it internally and discard it at
+ * completion of the merge.
+ * <p>
+ * The merge sorter only makes sense when we have at least one row. The
+ * caller must handle the special case of no rows.
+ */
+
+public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeSortWrapper.class);
+
+  public enum State { FIRST, BODY, EOF }
+
+  private SortRecordBatchBuilder builder;
+  private MSorter mSorter;
+  private SelectionVector4 sv4;
+  private int batchCount;
+  private State state = State.FIRST;
+  private final VectorContainer destContainer;
+
+  public MergeSortWrapper(OperExecContext opContext, VectorContainer destContainer) {
+    super(opContext);
+    this.destContainer = destContainer;
+  }
+
+  /**
+   * Merge the set of in-memory batches to produce a single logical output in the given
+   * destination container, indexed by an SV4.
+   *
+   * @param batchGroups the complete set of in-memory batches
+   * @param batch the record batch (operator) for the sort operator
+   * @param destContainer the vector container for the sort operator
+   * @return the sv4 for this operator
+   */
+
+  public void merge(List<BatchGroup.InputBatch> batchGroups) {
+
+    // Add the buffered batches to a collection that MSorter can use.
+    // The builder takes ownership of the batches and will release them if
+    // an error occurs.
+
+    builder = new SortRecordBatchBuilder(context.getAllocator());
+    for (BatchGroup.InputBatch group : batchGroups) {
+      RecordBatchData rbd = new RecordBatchData(group.getContainer(), context.getAllocator());
+      rbd.setSv2(group.getSv2());
+      builder.add(rbd);
+    }
+    batchGroups.clear();
+
+    // Generate the msorter.
+
+    try {
+      builder.build(destContainer);
+      sv4 = builder.getSv4();
+      Sort popConfig = context.getOperatorDefn();
+      mSorter = createNewMSorter(popConfig.getOrderings(), MAIN_MAPPING, LEFT_MAPPING, RIGHT_MAPPING);
+      mSorter.setup(context, context.getAllocator(), 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
+    context.injectUnchecked(ExternalSortBatch.INTERRUPTION_AFTER_SETUP);
+    mSorter.sort();
+
+    // For testing memory-leak purpose, inject exception after mSorter finishes sorting
+    context.injectUnchecked(ExternalSortBatch.INTERRUPTION_AFTER_SORT);
+    sv4 = mSorter.getSV4();
+
+    destContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
+  }
+
+  private MSorter createNewMSorter(List<Ordering> orderings, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping) {
+    CodeGenerator<MSorter> cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptionSet());
+    cg.plainJavaCapable(true);
+
+    // Uncomment out this line to debug the generated code.
+//  cg.saveCodeForDebugging(true);
+    ClassGenerator<MSorter> g = cg.getRoot();
+    g.setMappingSet(mainMapping);
+
+    for (Ordering od : orderings) {
+      // first, we rewrite the evaluation stack for each side of the comparison.
+      ErrorCollector collector = new ErrorCollectorImpl();
+      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), destContainer, collector, context.getFunctionRegistry());
+      if (collector.hasErrors()) {
+        throw UserException.unsupportedError()
+              .message("Failure while materializing expression. " + collector.toErrorString())
+              .build(logger);
+      }
+      g.setMappingSet(leftMapping);
+      HoldingContainer left = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
+      g.setMappingSet(rightMapping);
+      HoldingContainer right = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
+      g.setMappingSet(mainMapping);
+
+      // next we wrap the two comparison sides and add the expression block for the comparison.
+      LogicalExpression fh =
+          FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right,
+                                                         context.getFunctionRegistry());
+      HoldingContainer out = g.addExpr(fh, ClassGenerator.BlkCreateMode.FALSE);
+      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
+
+      if (od.getDirection() == Direction.ASCENDING) {
+        jc._then()._return(out.getValue());
+      }else{
+        jc._then()._return(out.getValue().minus());
+      }
+      g.rotateBlock();
+    }
+
+    g.rotateBlock();
+    g.getEvalBlock()._return(JExpr.lit(0));
+
+    return getInstance(cg, logger);
+  }
+
+  /**
+   * The SV4 provides a built-in iterator that returns a virtual set of record
+   * batches so that the downstream operator need not consume the entire set
+   * of accumulated batches in a single step.
+   */
+
+  @Override
+  public boolean next() {
+    switch (state) {
+    case BODY:
+      if (! sv4.next()) {
+        state = State.EOF;
+        return false;
+      }
+      return true;
+    case EOF:
+      return false;
+    case FIRST:
+      state = State.BODY;
+      return true;
+    default:
+      throw new IllegalStateException( "Unexpected case: " + state );
+    }
+  }
+
+  @Override
+  public void close() {
+    RuntimeException ex = null;
+    try {
+      if (builder != null) {
+        builder.clear();
+        builder.close();
+        builder = null;
+      }
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    try {
+      if (mSorter != null) {
+        mSorter.clear();
+        mSorter = null;
+      }
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    try {
+      if (sv4 != null) {
+        sv4.clear();
+      }
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    if (ex != null) {
+      throw ex;
+    }
+  }
+
+  @Override
+  public int getBatchCount() { return batchCount; }
+
+  @Override
+  public int getRecordCount() { return sv4.getCount(); }
+
+  @Override
+  public SelectionVector4 getSv4() { return sv4; }
+
+  @Override
+  public SelectionVector2 getSv2() { return null; }
+
+  @Override
+  public VectorContainer getContainer() { return destContainer; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
deleted file mode 100644
index 57846db..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/OperatorCodeGenerator.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * 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.
- * <p>
- * 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;
-
-  /**
-   * 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 context, Sort popConfig) {
-    this.context = context;
-    this.popConfig = popConfig;
-  }
-
-  public void setSchema(BatchSchema schema) {
-    close();
-    this.schema = schema;
-  }
-
-  public void close() {
-    closeCopier();
-    sorter = null;
-  }
-
-  public void closeCopier() {
-    if (copier == null) {
-      return; }
-    try {
-      copier.close();
-      copier = null;
-    } catch (IOException e) {
-      throw UserException.dataWriteError(e)
-            .message("Failure while flushing spilled data")
-            .build(logger);
-    }
-  }
-
-  public PriorityQueueCopier getCopier(VectorAccessible batch) {
-    if (copier == null) {
-      copier = generateCopier(batch);
-    }
-    return copier;
-  }
-
-  private PriorityQueueCopier generateCopier(VectorAccessible batch) {
-    // Generate the copier code and obtain the resulting class
-
-    CodeGenerator<PriorityQueueCopier> cg = CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
-    ClassGenerator<PriorityQueueCopier> g = cg.getRoot();
-    cg.plainJavaCapable(true);
-    // Uncomment out this line to debug the generated code.
-//  cg.saveCodeForDebugging(true);
-
-    generateComparisons(g, batch);
-
-    g.setMappingSet(COPIER_MAPPING_SET);
-    CopyUtil.generateCopies(g, batch, true);
-    g.setMappingSet(MAIN_MAPPING);
-    return getInstance(cg);
-  }
-
-  public MSorter createNewMSorter(VectorAccessible batch) {
-    return createNewMSorter(popConfig.getOrderings(), batch, MAIN_MAPPING, LEFT_MAPPING, RIGHT_MAPPING);
-  }
-
-  private MSorter createNewMSorter(List<Ordering> orderings, VectorAccessible batch, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping) {
-    CodeGenerator<MSorter> cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
-    cg.plainJavaCapable(true);
-
-    // Uncomment out this line to debug the generated code.
-//  cg.saveCodeForDebugging(true);
-    ClassGenerator<MSorter> g = cg.getRoot();
-    g.setMappingSet(mainMapping);
-
-    for (Ordering od : orderings) {
-      // first, we rewrite the evaluation stack for each side of the comparison.
-      ErrorCollector collector = new ErrorCollectorImpl();
-      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry());
-      if (collector.hasErrors()) {
-        throw UserException.unsupportedError()
-              .message("Failure while materializing expression. " + collector.toErrorString())
-              .build(logger);
-      }
-      g.setMappingSet(leftMapping);
-      HoldingContainer left = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
-      g.setMappingSet(rightMapping);
-      HoldingContainer right = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
-      g.setMappingSet(mainMapping);
-
-      // next we wrap the two comparison sides and add the expression block for the comparison.
-      LogicalExpression fh =
-          FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right,
-                                                         context.getFunctionRegistry());
-      HoldingContainer out = g.addExpr(fh, ClassGenerator.BlkCreateMode.FALSE);
-      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
-
-      if (od.getDirection() == Direction.ASCENDING) {
-        jc._then()._return(out.getValue());
-      }else{
-        jc._then()._return(out.getValue().minus());
-      }
-      g.rotateBlock();
-    }
-
-    g.rotateBlock();
-    g.getEvalBlock()._return(JExpr.lit(0));
-
-    return getInstance(cg);
-  }
-
-  public SingleBatchSorter getSorter(VectorAccessible batch) {
-    if (sorter == null) {
-      sorter = createNewSorter(batch);
-    }
-    return sorter;
-  }
-
-  private SingleBatchSorter createNewSorter(VectorAccessible batch) {
-    CodeGenerator<SingleBatchSorter> cg = CodeGenerator.get(
-        SingleBatchSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
-        context.getOptions());
-    ClassGenerator<SingleBatchSorter> g = cg.getRoot();
-    cg.plainJavaCapable(true);
-    // Uncomment out this line to debug the generated code.
-//  cg.saveCodeForDebugging(true);
-
-    generateComparisons(g, batch);
-    return getInstance(cg);
-  }
-
-  private <T> T getInstance(CodeGenerator<T> cg) {
-    try {
-      return context.getImplementationClass(cg);
-    } catch (ClassTransformationException e) {
-      throw UserException.unsupportedError(e)
-            .message("Code generation error - likely code error.")
-            .build(logger);
-    } catch (IOException e) {
-      throw UserException.resourceError(e)
-            .message("IO Error during code generation.")
-            .build(logger);
-    }
-  }
-
-  protected void generateComparisons(ClassGenerator<?> g, VectorAccessible batch)  {
-    g.setMappingSet(MAIN_MAPPING);
-
-    for (Ordering od : popConfig.getOrderings()) {
-      // first, we rewrite the evaluation stack for each side of the comparison.
-      ErrorCollector collector = new ErrorCollectorImpl();
-      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry());
-      if (collector.hasErrors()) {
-        throw UserException.unsupportedError()
-              .message("Failure while materializing expression. " + collector.toErrorString())
-              .build(logger);
-      }
-      g.setMappingSet(LEFT_MAPPING);
-      HoldingContainer left = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
-      g.setMappingSet(RIGHT_MAPPING);
-      HoldingContainer right = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
-      g.setMappingSet(MAIN_MAPPING);
-
-      // next we wrap the two comparison sides and add the expression block for the comparison.
-      LogicalExpression fh =
-          FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right,
-                                                         context.getFunctionRegistry());
-      HoldingContainer out = g.addExpr(fh, ClassGenerator.BlkCreateMode.FALSE);
-      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
-
-      if (od.getDirection() == Direction.ASCENDING) {
-        jc._then()._return(out.getValue());
-      }else{
-        jc._then()._return(out.getValue().minus());
-      }
-      g.rotateBlock();
-    }
-
-    g.rotateBlock();
-    g.getEvalBlock()._return(JExpr.lit(0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
index 2657bb8..be88232 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopier.java
@@ -23,11 +23,10 @@ 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 void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch,
+  public void setup(BufferAllocator allocator, VectorAccessible hyperBatch,
       List<BatchGroup> batchGroups, VectorAccessible outgoing) throws SchemaChangeException;
 
   public int next(int targetRecordCount);


[2/5] drill git commit: DRILL-5325: Unit tests for the managed sort

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
new file mode 100644
index 0000000..363c08c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -0,0 +1,216 @@
+/*
+ * 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.cache;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestBatchSerialization extends DrillTest {
+
+  public static OperatorFixture fixture;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    fixture = OperatorFixture.builder().build();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    fixture.close();
+  }
+
+  public SingleRowSet makeRowSet(BatchSchema schema, int rowCount) {
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer(rowCount);
+    for (int i = 0; i < rowCount; i++) {
+      RowSetUtilities.setFromInt(writer, 0, i);
+      writer.save();
+    }
+    writer.done();
+    return rowSet;
+  }
+
+  public SingleRowSet makeNullableRowSet(BatchSchema schema, int rowCount) {
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer(rowCount);
+    for (int i = 0; i < rowCount; i++) {
+      if (i % 2 == 0) {
+        RowSetUtilities.setFromInt(writer, 0, i);
+      } else {
+        writer.column(0).setNull();
+      }
+      writer.save();
+    }
+    writer.done();
+    return rowSet;
+  }
+
+  public void testType(MinorType type) throws IOException {
+    testNonNullType(type);
+    testNullableType(type);
+  }
+
+  public void testNonNullType(MinorType type) throws IOException {
+    BatchSchema schema = new SchemaBuilder( )
+        .add("col", type)
+        .build();
+    int rowCount = 20;
+    verifySerialize(makeRowSet(schema, rowCount),
+                    makeRowSet(schema, rowCount));
+  }
+
+  public void testNullableType(MinorType type) throws IOException {
+    BatchSchema schema = new SchemaBuilder( )
+        .addNullable("col", type)
+        .build();
+    int rowCount = 20;
+    verifySerialize(makeNullableRowSet(schema, rowCount),
+                    makeNullableRowSet(schema, rowCount));
+  }
+
+  /**
+   * Verify serialize and deserialize. Need to pass both the
+   * input and expected (even though the expected should be the same
+   * data as the input) because the act of serializing clears the
+   * input for obscure historical reasons.
+   *
+   * @param rowSet
+   * @param expected
+   * @throws IOException
+   */
+  private void verifySerialize(SingleRowSet rowSet, SingleRowSet expected) throws IOException {
+
+    long origSize = rowSet.size();
+
+    File dir = OperatorFixture.getTempDir("serial");
+    File outFile = new File(dir, "serialze.dat");
+    try (OutputStream out = new BufferedOutputStream(new FileOutputStream(outFile))) {
+      VectorSerializer.writer(fixture.allocator(), out)
+        .write(rowSet.container(), rowSet.getSv2());
+    }
+
+    RowSet result;
+    try (InputStream in = new BufferedInputStream(new FileInputStream(outFile))) {
+      result = fixture.wrap(
+        VectorSerializer.reader(fixture.allocator(), in)
+          .read());
+    }
+
+    assertTrue(origSize >= result.size());
+    new RowSetComparison(expected)
+      .verifyAndClear(result);
+    outFile.delete();
+  }
+
+  @Test
+  public void testTypes() throws IOException {
+    testType(MinorType.TINYINT);
+    testType(MinorType.UINT1);
+    testType(MinorType.SMALLINT);
+    testType(MinorType.UINT2);
+    testType(MinorType.INT);
+    testType(MinorType.UINT4);
+    testType(MinorType.BIGINT);
+    testType(MinorType.UINT8);
+    testType(MinorType.FLOAT4);
+    testType(MinorType.FLOAT8);
+    testType(MinorType.DECIMAL9);
+    testType(MinorType.DECIMAL18);
+    testType(MinorType.DECIMAL28SPARSE);
+    testType(MinorType.DECIMAL38SPARSE);
+//  testType(MinorType.DECIMAL28DENSE); No writer
+//  testType(MinorType.DECIMAL38DENSE); No writer
+    testType(MinorType.DATE);
+    testType(MinorType.TIME);
+    testType(MinorType.TIMESTAMP);
+    testType(MinorType.INTERVAL);
+    testType(MinorType.INTERVALYEAR);
+    testType(MinorType.INTERVALDAY);
+  }
+
+  private SingleRowSet buildMapSet(BatchSchema schema) {
+    return fixture.rowSetBuilder(schema)
+        .add(1, 100, "first")
+        .add(2, 200, "second")
+        .add(3, 300, "third")
+        .build();
+  }
+
+  private SingleRowSet buildArraySet(BatchSchema schema) {
+    return fixture.rowSetBuilder(schema)
+        .add(1, new String[] { "first, second, third" } )
+        .add(2, null)
+        .add(3, new String[] { "third, fourth, fifth" } )
+        .build();
+  }
+
+  /**
+   * Tests a map type and an SV2.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testMap() throws IOException {
+    BatchSchema schema = new SchemaBuilder()
+        .add("top", MinorType.INT)
+        .addMap("map")
+          .add("key", MinorType.INT)
+          .add("value", MinorType.VARCHAR)
+          .buildMap()
+        .build();
+
+    verifySerialize(buildMapSet(schema).toIndirect(),
+                    buildMapSet(schema));
+  }
+
+  @Test
+  public void testArray() throws IOException {
+    BatchSchema schema = new SchemaBuilder()
+        .add("top", MinorType.INT)
+        .addArray("arr", MinorType.VARCHAR)
+        .build();
+
+    verifySerialize(buildArraySet(schema).toIndirect(),
+                    buildArraySet(schema));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
index 96dae6a..2473dc5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
@@ -91,6 +91,7 @@ public class TestWriteToDisk extends ExecTest {
         VectorContainer container = new VectorContainer();
         container.addCollection(vectorList);
         container.setRecordCount(4);
+        @SuppressWarnings("resource")
         WritableBatch batch = WritableBatch.getBatchNoHVWrap(
             container.getRecordCount(), container, false);
         VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(
@@ -107,7 +108,6 @@ public class TestWriteToDisk extends ExecTest {
           final Path path = new Path(tempDir.getAbsolutePath(), "drillSerializable");
           try (final FSDataOutputStream out = fs.create(path)) {
             wrap.writeToStream(out);
-            out.close();
           }
 
           try (final FSDataInputStream in = fs.open(path)) {

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
index 52ebd57..6ead748 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
@@ -20,13 +20,16 @@ package org.apache.drill.exec.physical.impl.xsort;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.TestBuilder;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.SecondaryTest;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 
+@Category(SecondaryTest.class)
 public class TestExternalSort extends BaseTestQuery {
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index 50bf710..f643d5f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -35,7 +35,6 @@ import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.FixtureBuilder;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
@@ -46,6 +45,11 @@ public class TestSimpleExternalSort extends DrillTest {
   @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(80000);
 
   @Test
+  public void mergeSortWithSv2Managed() throws Exception {
+    mergeSortWithSv2(false);
+  }
+
+  @Test
   public void mergeSortWithSv2Legacy() throws Exception {
     mergeSortWithSv2(true);
   }
@@ -62,26 +66,37 @@ public class TestSimpleExternalSort extends DrillTest {
    */
 
   private void mergeSortWithSv2(boolean testLegacy) throws Exception {
-    try (ClusterFixture cluster = ClusterFixture.standardCluster( );
+    FixtureBuilder builder = ClusterFixture.builder()
+        .configProperty(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED, false)
+         ;
+    try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
       chooseImpl(client, testLegacy);
       List<QueryDataBatch> results = client.queryBuilder().physicalResource("xsort/one_key_sort_descending_sv2.json").results();
-      assertEquals(500000, client.countResults( results ));
+      assertEquals(500_000, client.countResults(results));
       validateResults(client.allocator(), results);
     }
   }
 
   private void chooseImpl(ClientFixture client, boolean testLegacy) throws Exception {
+    client.alterSession(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED_OPTION.getOptionName(), testLegacy);
+  }
+
+  @Test
+  public void sortOneKeyDescendingMergeSortManaged() throws Throwable {
+    sortOneKeyDescendingMergeSort(false);
   }
 
   @Test
-  @Ignore
   public void sortOneKeyDescendingMergeSortLegacy() throws Throwable {
     sortOneKeyDescendingMergeSort(true);
   }
 
   private void sortOneKeyDescendingMergeSort(boolean testLegacy) throws Throwable {
-    try (ClusterFixture cluster = ClusterFixture.standardCluster( );
+    FixtureBuilder builder = ClusterFixture.builder()
+        .configProperty(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED, false)
+         ;
+    try (ClusterFixture cluster = builder.build();
          ClientFixture client = cluster.clientFixture()) {
       chooseImpl(client, testLegacy);
       List<QueryDataBatch> results = client.queryBuilder().physicalResource("xsort/one_key_sort_descending.json").results();
@@ -101,7 +116,7 @@ public class TestSimpleExternalSort extends DrillTest {
       if (b.getHeader().getRowCount() > 0) {
         batchCount++;
         loader.load(b.getHeader().getDef(),b.getData());
-        @SuppressWarnings("resource")
+        @SuppressWarnings({ "deprecation", "resource" })
         BigIntVector c1 = (BigIntVector) loader.getValueAccessorById(BigIntVector.class, loader.getValueVectorId(new SchemaPath("blue", ExpressionPosition.UNKNOWN)).getFieldIds()).getValueVector();
         BigIntVector.Accessor a1 = c1.getAccessor();
 
@@ -118,44 +133,56 @@ public class TestSimpleExternalSort extends DrillTest {
     System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));
   }
 
+  @Test
+  public void sortOneKeyDescendingExternalSortManaged() throws Throwable {
+    sortOneKeyDescendingExternalSort(false);
+  }
 
   @Test
-  @Ignore
   public void sortOneKeyDescendingExternalSortLegacy() throws Throwable {
     sortOneKeyDescendingExternalSort(true);
   }
 
   private void sortOneKeyDescendingExternalSort(boolean testLegacy) throws Throwable {
-    FixtureBuilder builder = ClusterFixture.builder( )
-        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD, 4 )
-        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE, 4);
+    FixtureBuilder builder = ClusterFixture.builder()
+        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD, 4)
+        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE, 4)
+        .configProperty(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 4)
+        .configProperty(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED, false)
+        ;
     try (ClusterFixture cluster = builder.build();
-        ClientFixture client = cluster.clientFixture()) {
+         ClientFixture client = cluster.clientFixture()) {
       chooseImpl(client,testLegacy);
       List<QueryDataBatch> results = client.queryBuilder().physicalResource("/xsort/one_key_sort_descending.json").results();
-      assertEquals(1000000, client.countResults( results ));
+      assertEquals(1_000_000, client.countResults(results));
       validateResults(client.allocator(), results);
     }
   }
 
-  @Ignore
+  @Test
+  public void outOfMemoryExternalSortManaged() throws Throwable{
+    outOfMemoryExternalSort(false);
+  }
+
   @Test
   public void outOfMemoryExternalSortLegacy() throws Throwable{
     outOfMemoryExternalSort(true);
   }
 
   private void outOfMemoryExternalSort(boolean testLegacy) throws Throwable{
-    FixtureBuilder builder = ClusterFixture.builder( )
+    FixtureBuilder builder = ClusterFixture.builder()
         // Probably do nothing in modern Drill
-        .configProperty( "drill.memory.fragment.max", 50000000 )
-        .configProperty( "drill.memory.fragment.initial", 2000000 )
-        .configProperty( "drill.memory.operator.max", 30000000 )
-        .configProperty( "drill.memory.operator.initial", 2000000 );
+        .configProperty("drill.memory.fragment.max", 50_000_000)
+        .configProperty("drill.memory.fragment.initial", 2_000_000)
+        .configProperty("drill.memory.operator.max", 30_000_000)
+        .configProperty("drill.memory.operator.initial", 2_000_000)
+        .configProperty(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED, testLegacy)
+        ;
     try (ClusterFixture cluster = builder.build();
-        ClientFixture client = cluster.clientFixture()) {
+         ClientFixture client = cluster.clientFixture()) {
       chooseImpl(client,testLegacy);
       List<QueryDataBatch> results = client.queryBuilder().physicalResource("/xsort/oom_sort_test.json").results();
-      assertEquals(10000000, client.countResults( results ));
+      assertEquals(10_000_000, client.countResults(results));
 
       long previousBigInt = Long.MAX_VALUE;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
index 788caf7..5a1bf6d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSortSpillWithException.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,22 +17,23 @@
  */
 package org.apache.drill.exec.physical.impl.xsort;
 
-import org.apache.drill.BaseTestQuery;
-import org.apache.drill.common.config.DrillConfig;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.testing.Controls;
 import org.apache.drill.exec.testing.ControlsInjectionUtil;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.FixtureBuilder;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Properties;
 
 /**
  * Testing External Sort's spilling to disk.
@@ -42,30 +43,33 @@ import java.util.Properties;
  * <br>
  * {@link ExecConstants#EXTERNAL_SORT_SPILL_GROUP_SIZE} = 1
  */
-public class TestSortSpillWithException extends BaseTestQuery {
+public class TestSortSpillWithException extends ClusterTest {
   private static final String TEST_RES_PATH = TestTools.getWorkingPath() + "/src/test/resources";
 
   @BeforeClass
-  public static void initCluster() {
-    // make sure memory sorter outputs 20 rows per batch
-    final Properties props = cloneDefaultTestConfigProperties();
-    props.put(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD, "1");
-    props.put(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE, "1");
-
-    updateTestCluster(1, DrillConfig.create(props));
+  public static void setup() throws Exception {
+    FixtureBuilder builder = ClusterFixture.builder()
+        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD, 1) // Unmanaged
+        .configProperty(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE, 1) // Unmanaged
+        .sessionOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE_KEY, 60 * 1024 * 1024) // Spill early
+        .configProperty(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED, false)
+        .maxParallelization(1)
+        ;
+    startCluster(builder);
   }
 
   @Test
-  public void testSpilLeak() throws Exception {
+  public void testSpillLeakLegacy() throws Exception {
+    client.alterSession(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED_OPTION.getOptionName(), true);
     // inject exception in sort while spilling
     final String controls = Controls.newBuilder()
       .addExceptionOnBit(
-          ExternalSortBatch.class,
-          ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
+          org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.class,
+          org.apache.drill.exec.physical.impl.xsort.ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
           IOException.class,
-          bits[0].getContext().getEndpoint())
+          cluster.drillbit().getContext().getEndpoint())
       .build();
-    ControlsInjectionUtil.setControls(client, controls);
+    ControlsInjectionUtil.setControls(cluster.client(), controls);
     // run a simple order by query
     try {
       test("select employee_id from dfs_test.`%s/xsort/2batches` order by employee_id", TEST_RES_PATH);
@@ -76,4 +80,28 @@ public class TestSortSpillWithException extends BaseTestQuery {
         e.getMessage().contains("External Sort encountered an error while spilling to disk"));
     }
   }
+
+  @Test
+  public void testSpillLeakManaged() throws Exception {
+    client.alterSession(ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED_OPTION.getOptionName(), false);
+    // inject exception in sort while spilling
+    final String controls = Controls.newBuilder()
+      .addExceptionOnBit(
+          org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.class,
+          org.apache.drill.exec.physical.impl.xsort.managed.ExternalSortBatch.INTERRUPTION_WHILE_SPILLING,
+          IOException.class,
+          cluster.drillbit().getContext().getEndpoint())
+      .build();
+    ControlsInjectionUtil.setControls(cluster.client(), controls);
+    // run a simple order by query
+    try {
+      test("SELECT id_i, name_s250 FROM `mock`.`employee_500K` ORDER BY id_i");
+//      test("select employee_id from dfs_test.`%s/xsort/2batches` order by employee_id", TEST_RES_PATH);
+      fail("Query should have failed!");
+    } catch (UserRemoteException e) {
+      assertEquals(ErrorType.RESOURCE, e.getErrorType());
+      assertTrue("Incorrect error message",
+        e.getMessage().contains("External Sort encountered an error while spilling to disk"));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
new file mode 100644
index 0000000..034da2c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -0,0 +1,132 @@
+/*
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrapper.BatchMerger;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetSchema;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+
+import com.google.common.collect.Lists;
+
+public class SortTestUtilities {
+
+  private SortTestUtilities() { }
+
+  public static BatchSchema makeSchema(MinorType type, boolean nullable) {
+    return new SchemaBuilder()
+        .add("key", type, nullable ? DataMode.OPTIONAL : DataMode.REQUIRED)
+        .add("value", MinorType.VARCHAR)
+        .build();
+  }
+
+  public static BatchSchema nonNullSchema() {
+    return makeSchema(MinorType.INT, false);
+  }
+
+  public static BatchSchema nullableSchema() {
+    return makeSchema(MinorType.INT, true);
+  }
+
+  public static PriorityQueueCopierWrapper makeCopier(OperatorFixture fixture, String sortOrder, String nullOrder) {
+    FieldReference expr = FieldReference.getWithQuotedRef("key");
+    Ordering ordering = new Ordering(sortOrder, expr, nullOrder);
+    Sort popConfig = new Sort(null, Lists.newArrayList(ordering), false);
+    OperExecContext opContext = fixture.newOperExecContext(popConfig);
+    return new PriorityQueueCopierWrapper(opContext);
+  }
+
+  public static class CopierTester {
+    List<SingleRowSet> rowSets = new ArrayList<>();
+    List<SingleRowSet> expected = new ArrayList<>();
+    String sortOrder = Ordering.ORDER_ASC;
+    String nullOrder = Ordering.NULLS_UNSPECIFIED;
+    private OperatorFixture fixture;
+
+    public CopierTester(OperatorFixture fixture) {
+      this.fixture = fixture;
+    }
+
+    public void addInput(SingleRowSet input) {
+      rowSets.add(input);
+    }
+
+    public void addOutput(SingleRowSet output) {
+      expected.add(output);
+    }
+
+    public void run() throws Exception {
+      PriorityQueueCopierWrapper copier = makeCopier(fixture, sortOrder, nullOrder);
+      List<BatchGroup> batches = new ArrayList<>();
+      RowSetSchema schema = null;
+      for (SingleRowSet rowSet : rowSets) {
+        batches.add(new BatchGroup.InputBatch(rowSet.container(), rowSet.getSv2(),
+                    fixture.allocator(), rowSet.size()));
+        if (schema == null) {
+          schema = rowSet.schema();
+        }
+      }
+      int rowCount = outputRowCount();
+      VectorContainer dest = new VectorContainer();
+      @SuppressWarnings("resource")
+      BatchMerger merger = copier.startMerge(schema.toBatchSchema(SelectionVectorMode.NONE),
+                                             batches, dest, rowCount);
+
+      verifyResults(merger, dest);
+      dest.clear();
+      merger.close();
+    }
+
+    public int outputRowCount() {
+      if (! expected.isEmpty()) {
+        return expected.get(0).rowCount();
+      }
+      return 10;
+    }
+
+    protected void verifyResults(BatchMerger merger, VectorContainer dest) {
+      for (RowSet expectedSet : expected) {
+        assertTrue(merger.next());
+        RowSet rowSet = new DirectRowSet(fixture.allocator(), dest);
+        new RowSetComparison(expectedSet)
+              .verifyAndClear(rowSet);
+      }
+      assertFalse(merger.next());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
new file mode 100644
index 0000000..0050747
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
@@ -0,0 +1,377 @@
+/*
+ * 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 static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrapper.BatchMerger;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortTestUtilities.CopierTester;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Light-weight sanity test of the copier class. The implementation has
+ * been used in production, so the tests here just check for the obvious
+ * cases.
+ * <p>
+ * Note, however, that if significant changes are made to the copier,
+ * then additional tests should be added to re-validate the code.
+ */
+
+public class TestCopier extends DrillTest {
+
+  public static OperatorFixture fixture;
+
+  @BeforeClass
+  public static void setup() {
+    fixture = OperatorFixture.builder().build();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    fixture.close();
+  }
+
+  @Test
+  public void testEmptyInput() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    List<BatchGroup> batches = new ArrayList<>();
+    PriorityQueueCopierWrapper copier = SortTestUtilities.makeCopier(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED);
+    VectorContainer dest = new VectorContainer();
+    try {
+      @SuppressWarnings({ "resource", "unused" })
+      BatchMerger merger = copier.startMerge(schema, batches, dest, 10);
+      fail();
+    } catch (AssertionError e) {
+      // Expected
+    }
+  }
+
+  @Test
+  public void testEmptyBatch() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(fixture.rowSetBuilder(schema)
+          .withSv2()
+          .build());
+
+    tester.run();
+  }
+
+  @Test
+  public void testSingleRow() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(fixture.rowSetBuilder(schema)
+          .add(10, "10")
+          .withSv2()
+          .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+          .add(10, "10")
+          .build());
+    tester.run();
+  }
+
+  @Test
+  public void testTwoBatchesSingleRow() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(fixture.rowSetBuilder(schema)
+          .add(10, "10")
+          .withSv2()
+          .build());
+    tester.addInput(fixture.rowSetBuilder(schema)
+          .add(20, "20")
+          .withSv2()
+          .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+          .add(10, "10")
+          .add(20, "20")
+          .build());
+    tester.run();
+  }
+
+  public static SingleRowSet makeDataSet(BatchSchema schema, int first, int step, int count) {
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer(count);
+    int value = first;
+    for (int i = 0; i < count; i++, value += step) {
+      RowSetUtilities.setFromInt(writer, 0, value);
+      writer.column(1).setString(Integer.toString(value));
+      writer.save();
+    }
+    writer.done();
+    return rowSet;
+  }
+
+  @Test
+  public void testMultipleOutput() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(makeDataSet(schema, 0, 2, 10).toIndirect());
+    tester.addInput(makeDataSet(schema, 1, 2, 10).toIndirect());
+
+    tester.addOutput(makeDataSet(schema, 0, 1, 10));
+    tester.addOutput(makeDataSet(schema, 10, 1, 10));
+    tester.run();
+  }
+
+  // Also verifies that SV2s work
+
+  @Test
+  public void testMultipleOutputDesc() throws Exception {
+    BatchSchema schema = SortTestUtilities.nonNullSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.sortOrder = Ordering.ORDER_DESC;
+    tester.nullOrder = Ordering.NULLS_UNSPECIFIED;
+    SingleRowSet input = makeDataSet(schema, 0, 2, 10).toIndirect();
+    RowSetUtilities.reverse(input.getSv2());
+    tester.addInput(input);
+
+    input = makeDataSet(schema, 1, 2, 10).toIndirect();
+    RowSetUtilities.reverse(input.getSv2());
+    tester.addInput(input);
+
+    tester.addOutput(makeDataSet(schema, 19, -1, 10));
+    tester.addOutput(makeDataSet(schema, 9, -1, 10));
+
+    tester.run();
+  }
+
+  @Test
+  public void testAscNullsLast() throws Exception {
+    BatchSchema schema = SortTestUtilities.nullableSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.sortOrder = Ordering.ORDER_ASC;
+    tester.nullOrder = Ordering.NULLS_LAST;
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(1, "1")
+        .add(4, "4")
+        .add(null, "null")
+        .withSv2()
+        .build());
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(2, "2")
+        .add(3, "3")
+        .add(null, "null")
+        .withSv2()
+        .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+        .add(1, "1")
+        .add(2, "2")
+        .add(3, "3")
+        .add(4, "4")
+        .add(null, "null")
+        .add(null, "null")
+        .build());
+
+    tester.run();
+  }
+
+  @Test
+  public void testAscNullsFirst() throws Exception {
+    BatchSchema schema = SortTestUtilities.nullableSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.sortOrder = Ordering.ORDER_ASC;
+    tester.nullOrder = Ordering.NULLS_FIRST;
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(1, "1")
+        .add(4, "4")
+        .withSv2()
+        .build());
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(2, "2")
+        .add(3, "3")
+        .withSv2()
+        .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(null, "null")
+        .add(1, "1")
+        .add(2, "2")
+        .add(3, "3")
+        .add(4, "4")
+        .build());
+
+    tester.run();
+  }
+
+  @Test
+  public void testDescNullsLast() throws Exception {
+    BatchSchema schema = SortTestUtilities.nullableSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.sortOrder = Ordering.ORDER_DESC;
+    tester.nullOrder = Ordering.NULLS_LAST;
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(4, "4")
+        .add(1, "1")
+        .add(null, "null")
+        .withSv2()
+        .build());
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(3, "3")
+        .add(2, "2")
+        .add(null, "null")
+        .withSv2()
+        .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+        .add(4, "4")
+        .add(3, "3")
+        .add(2, "2")
+        .add(1, "1")
+        .add(null, "null")
+        .add(null, "null")
+        .build());
+
+    tester.run();
+  }
+
+  @Test
+  public void testDescNullsFirst() throws Exception {
+    BatchSchema schema = SortTestUtilities.nullableSchema();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.sortOrder = Ordering.ORDER_DESC;
+    tester.nullOrder = Ordering.NULLS_FIRST;
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(4, "4")
+        .add(1, "1")
+        .withSv2()
+        .build());
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(3, "3")
+        .add(2, "2")
+        .withSv2()
+        .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+        .add(null, "null")
+        .add(null, "null")
+        .add(4, "4")
+        .add(3, "3")
+        .add(2, "2")
+        .add(1, "1")
+        .build());
+
+    tester.run();
+  }
+
+  public static void runTypeTest(OperatorFixture fixture, MinorType type) throws Exception {
+    BatchSchema schema = SortTestUtilities.makeSchema(type, false);
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(makeDataSet(schema, 0, 2, 5).toIndirect());
+    tester.addInput(makeDataSet(schema, 1, 2, 5).toIndirect());
+
+    tester.addOutput(makeDataSet(schema, 0, 1, 10));
+
+    tester.run();
+  }
+
+  @Test
+  public void testTypes() throws Exception {
+    testAllTypes(fixture);
+  }
+
+  public static void testAllTypes(OperatorFixture fixture) throws Exception {
+    runTypeTest(fixture, MinorType.INT);
+    runTypeTest(fixture, MinorType.BIGINT);
+    runTypeTest(fixture, MinorType.FLOAT4);
+    runTypeTest(fixture, MinorType.FLOAT8);
+    runTypeTest(fixture, MinorType.DECIMAL9);
+    runTypeTest(fixture, MinorType.DECIMAL18);
+    runTypeTest(fixture, MinorType.VARCHAR);
+    runTypeTest(fixture, MinorType.VARBINARY);
+    runTypeTest(fixture, MinorType.DATE);
+    runTypeTest(fixture, MinorType.TIME);
+    runTypeTest(fixture, MinorType.TIMESTAMP);
+    runTypeTest(fixture, MinorType.INTERVAL);
+    runTypeTest(fixture, MinorType.INTERVALDAY);
+    runTypeTest(fixture, MinorType.INTERVALYEAR);
+
+    // Others not tested. See DRILL-5329
+  }
+
+  @Test
+  public void testMapType() throws Exception {
+    testMapType(fixture);
+  }
+
+  public void testMapType(OperatorFixture fixture) throws Exception {
+    BatchSchema schema = new SchemaBuilder()
+        .add("key", MinorType.INT)
+        .addMap("m1")
+          .add("b", MinorType.INT)
+          .addMap("m2")
+            .add("c", MinorType.INT)
+            .buildMap()
+          .buildMap()
+        .build();
+
+    CopierTester tester = new CopierTester(fixture);
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(1, 10, 100)
+        .add(5, 50, 500)
+        .withSv2()
+        .build());
+
+    tester.addInput(fixture.rowSetBuilder(schema)
+        .add(2, 20, 200)
+        .add(6, 60, 600)
+        .withSv2()
+        .build());
+
+    tester.addOutput(fixture.rowSetBuilder(schema)
+        .add(1, 10, 100)
+        .add(2, 20, 200)
+        .add(5, 50, 500)
+        .add(6, 60, 600)
+        .build());
+
+    tester.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortExec.java
new file mode 100644
index 0000000..f5858a3
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortExec.java
@@ -0,0 +1,188 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.test.DrillTest;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestExternalSortExec extends DrillTest {
+
+  @Test
+  public void testFieldReference() {
+    // Misnomer: the reference must be unquoted.
+    FieldReference expr = FieldReference.getWithQuotedRef("foo");
+    assertEquals(Types.LATE_BIND_TYPE, expr.getMajorType());
+    assertTrue(expr.isSimplePath());
+    assertEquals("foo", expr.getRootSegment().getPath());
+    assertEquals("`foo`", expr.toExpr());
+  }
+
+  @Test
+  public void testOrdering() {
+    assertEquals(Direction.ASCENDING, Ordering.getOrderingSpecFromString(null));
+    assertEquals(Direction.ASCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_ASC));
+    assertEquals(Direction.DESCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_DESC));
+    assertEquals(Direction.ASCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_ASCENDING));
+    assertEquals(Direction.DESCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_DESCENDING));
+    assertEquals(Direction.ASCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_ASC.toLowerCase()));
+    assertEquals(Direction.DESCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_DESC.toLowerCase()));
+    assertEquals(Direction.ASCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_ASCENDING.toLowerCase()));
+    assertEquals(Direction.DESCENDING, Ordering.getOrderingSpecFromString(Ordering.ORDER_DESCENDING.toLowerCase()));
+    try {
+      Ordering.getOrderingSpecFromString("");
+      fail();
+    } catch(DrillRuntimeException e) { }
+    try {
+      Ordering.getOrderingSpecFromString("foo");
+      fail();
+    } catch(DrillRuntimeException e) { }
+
+    assertEquals(NullDirection.UNSPECIFIED, Ordering.getNullOrderingFromString(null));
+    assertEquals(NullDirection.FIRST, Ordering.getNullOrderingFromString(Ordering.NULLS_FIRST));
+    assertEquals(NullDirection.LAST, Ordering.getNullOrderingFromString(Ordering.NULLS_LAST));
+    assertEquals(NullDirection.UNSPECIFIED, Ordering.getNullOrderingFromString(Ordering.NULLS_UNSPECIFIED));
+    assertEquals(NullDirection.FIRST, Ordering.getNullOrderingFromString(Ordering.NULLS_FIRST.toLowerCase()));
+    assertEquals(NullDirection.LAST, Ordering.getNullOrderingFromString(Ordering.NULLS_LAST.toLowerCase()));
+    assertEquals(NullDirection.UNSPECIFIED, Ordering.getNullOrderingFromString(Ordering.NULLS_UNSPECIFIED.toLowerCase()));
+    try {
+      Ordering.getNullOrderingFromString("");
+      fail();
+    } catch(DrillRuntimeException e) { }
+    try {
+      Ordering.getNullOrderingFromString("foo");
+      fail();
+    } catch(DrillRuntimeException e) { }
+
+    FieldReference expr = FieldReference.getWithQuotedRef("foo");
+
+    // Test all getters
+
+    Ordering ordering = new Ordering((String) null, expr, (String) null);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+    assertSame(expr, ordering.getExpr());
+    assertTrue(ordering.nullsSortHigh());
+
+    // Test all ordering strings
+
+    ordering = new Ordering((String) Ordering.ORDER_ASC, expr, (String) null);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+
+    ordering = new Ordering((String) Ordering.ORDER_ASC.toLowerCase(), expr, (String) null);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+
+    ordering = new Ordering((String) Ordering.ORDER_ASCENDING, expr, (String) null);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+
+    ordering = new Ordering((String) Ordering.ORDER_DESC, expr, (String) null);
+    assertEquals(Direction.DESCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+
+    ordering = new Ordering((String) Ordering.ORDER_DESCENDING, expr, (String) null);
+    assertEquals(Direction.DESCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+
+    // Test all null ordering strings
+
+    ordering = new Ordering((String) null, expr, Ordering.NULLS_FIRST);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.FIRST, ordering.getNullDirection());
+    assertFalse(ordering.nullsSortHigh());
+
+    ordering = new Ordering((String) null, expr, Ordering.NULLS_FIRST);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.FIRST, ordering.getNullDirection());
+    assertFalse(ordering.nullsSortHigh());
+
+    ordering = new Ordering((String) null, expr, Ordering.NULLS_LAST);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.LAST, ordering.getNullDirection());
+    assertTrue(ordering.nullsSortHigh());
+
+    ordering = new Ordering((String) null, expr, Ordering.NULLS_UNSPECIFIED);
+    assertEquals(Direction.ASCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+    assertTrue(ordering.nullsSortHigh());
+
+    // Unspecified order is always nulls high
+
+    ordering = new Ordering(Ordering.ORDER_DESC, expr, Ordering.NULLS_UNSPECIFIED);
+    assertEquals(Direction.DESCENDING, ordering.getDirection());
+    assertEquals(NullDirection.UNSPECIFIED, ordering.getNullDirection());
+    assertTrue(ordering.nullsSortHigh());
+
+    // Null sort direction reverses with a Desc sort.
+
+    ordering = new Ordering(Ordering.ORDER_DESC, expr, Ordering.NULLS_FIRST);
+    assertEquals(Direction.DESCENDING, ordering.getDirection());
+    assertEquals(NullDirection.FIRST, ordering.getNullDirection());
+    assertTrue(ordering.nullsSortHigh());
+
+    ordering = new Ordering(Ordering.ORDER_DESC, expr, Ordering.NULLS_LAST);
+    assertEquals(Direction.DESCENDING, ordering.getDirection());
+    assertEquals(NullDirection.LAST, ordering.getNullDirection());
+    assertFalse(ordering.nullsSortHigh());
+  }
+
+  @Test
+  public void testSortSpec() {
+    FieldReference expr = FieldReference.getWithQuotedRef("foo");
+    Ordering ordering = new Ordering(Ordering.ORDER_ASC, expr, Ordering.NULLS_FIRST);
+
+    // Basics
+
+    ExternalSort popConfig = new ExternalSort(null, Lists.newArrayList(ordering), false);
+    assertSame(ordering, popConfig.getOrderings().get(0));
+    assertFalse(popConfig.getReverse());
+    assertEquals(SelectionVectorMode.FOUR_BYTE, popConfig.getSVMode());
+    assertEquals(CoreOperatorType.EXTERNAL_SORT_VALUE, popConfig.getOperatorType());
+    assertEquals(ExternalSort.DEFAULT_SORT_ALLOCATION, popConfig.getInitialAllocation());
+    assertEquals(AbstractBase.MAX_ALLOCATION, popConfig.getMaxAllocation());
+    assertTrue(popConfig.isExecutable());
+
+    // Non-default settings
+
+    popConfig = new ExternalSort(null, Lists.newArrayList(ordering), true);
+    assertTrue(popConfig.getReverse());
+    long maxAlloc = 50_000_000;
+    popConfig.setMaxAllocation(maxAlloc);
+    assertEquals(ExternalSort.DEFAULT_SORT_ALLOCATION, popConfig.getInitialAllocation());
+    assertEquals(maxAlloc, popConfig.getMaxAllocation());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
new file mode 100644
index 0000000..6bff088
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
@@ -0,0 +1,632 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeAction;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeTask;
+import org.apache.drill.test.ConfigBuilder;
+import org.apache.drill.test.DrillTest;
+import org.apache.drill.test.OperatorFixture;
+import org.junit.Test;
+
+public class TestExternalSortInternals extends DrillTest {
+
+  private static final int ONE_MEG = 1024 * 1024;
+
+  /**
+   * Verify defaults configured in drill-override.conf.
+   */
+  @Test
+  public void testConfigDefaults() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    // Zero means no artificial limit
+    assertEquals(0, sortConfig.maxMemory());
+    // Zero mapped to large number
+    assertEquals(Integer.MAX_VALUE, sortConfig.mergeLimit());
+    // Default size: 256 MiB
+    assertEquals(256 * ONE_MEG, sortConfig.spillFileSize());
+    // Default size: 8 MiB
+    assertEquals(8 * ONE_MEG, sortConfig.spillBatchSize());
+    // Default size: 16 MiB
+    assertEquals(16 * ONE_MEG, sortConfig.mergeBatchSize());
+    // Default: unlimited
+    assertEquals(Integer.MAX_VALUE, sortConfig.getBufferedBatchLimit());
+  }
+
+  /**
+   * Verify that the various constants do, in fact, map to the
+   * expected properties, and that the properties are overridden.
+   */
+  @Test
+  public void testConfigOverride() {
+    // Verify the various HOCON ways of setting memory
+    DrillConfig drillConfig = new ConfigBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MAX_MEMORY, "2000K")
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, 10)
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE, "10M")
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, 500_000)
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, 600_000)
+        .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 50)
+        .build();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    assertEquals(2000 * 1024, sortConfig.maxMemory());
+    assertEquals(10, sortConfig.mergeLimit());
+    assertEquals(10 * ONE_MEG, sortConfig.spillFileSize());
+    assertEquals(500_000, sortConfig.spillBatchSize());
+    assertEquals(600_000, sortConfig.mergeBatchSize());
+    assertEquals(50, sortConfig.getBufferedBatchLimit());
+  }
+
+  /**
+   * Some properties have hard-coded limits. Verify these limits.
+   */
+  @Test
+  public void testConfigLimits() {
+    DrillConfig drillConfig = new ConfigBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, SortConfig.MIN_MERGE_LIMIT - 1)
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE, SortConfig.MIN_SPILL_FILE_SIZE - 1)
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, SortConfig.MIN_SPILL_BATCH_SIZE - 1)
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, SortConfig.MIN_MERGE_BATCH_SIZE - 1)
+        .put(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT, 1)
+        .build();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    assertEquals(SortConfig.MIN_MERGE_LIMIT, sortConfig.mergeLimit());
+    assertEquals(SortConfig.MIN_SPILL_FILE_SIZE, sortConfig.spillFileSize());
+    assertEquals(SortConfig.MIN_SPILL_BATCH_SIZE, sortConfig.spillBatchSize());
+    assertEquals(SortConfig.MIN_MERGE_BATCH_SIZE, sortConfig.mergeBatchSize());
+    assertEquals(2, sortConfig.getBufferedBatchLimit());
+  }
+
+  @Test
+  public void testMemoryManagerBasics() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 50 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Basic setup
+
+    assertEquals(sortConfig.spillBatchSize(), memManager.getPreferredSpillBatchSize());
+    assertEquals(sortConfig.mergeBatchSize(), memManager.getPreferredMergeBatchSize());
+    assertEquals(memoryLimit, memManager.getMemoryLimit());
+
+    // Nice simple batch: 6 MB in size, 300 byte rows, vectors half full
+    // so 10000 rows. Sizes chosen so that spill and merge batch record
+    // stay below the limit of 64K.
+
+    int rowWidth = 300;
+    int rowCount = 10000;
+    int batchSize = rowWidth * rowCount * 2;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
+
+    // Zero rows - no update
+
+    memManager.updateEstimates(batchSize, rowWidth, 0);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Larger batch size, update batch size
+
+    rowCount = 20000;
+    batchSize = rowWidth * rowCount * 2;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
+
+    // Smaller batch size: no change
+
+    rowCount = 5000;
+    int lowBatchSize = rowWidth * rowCount * 2;
+    memManager.updateEstimates(lowBatchSize, rowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Different batch density, update batch size
+
+    rowCount = 10000;
+    batchSize = rowWidth * rowCount * 5;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
+
+    // Smaller row size, no update
+
+    int lowRowWidth = 200;
+    rowCount = 10000;
+    lowBatchSize = rowWidth * rowCount * 2;
+    memManager.updateEstimates(lowBatchSize, lowRowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Larger row size, updates calcs
+
+    rowWidth = 400;
+    rowCount = 10000;
+    lowBatchSize = rowWidth * rowCount * 2;
+    memManager.updateEstimates(lowBatchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memoryLimit, memManager, batchSize, rowWidth, rowCount);
+
+    // EOF: very low density
+
+    memManager.updateEstimates(lowBatchSize, rowWidth, 5);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+  }
+
+  private void verifyCalcs(SortConfig sortConfig, long memoryLimit, SortMemoryManager memManager, int batchSize,
+      int rowWidth, int rowCount) {
+
+    assertFalse(memManager.mayOverflow());
+
+    // Row and batch sizes should be exact
+
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Spill sizes will be rounded, but within reason.
+
+    int count = sortConfig.spillBatchSize() / rowWidth;
+    assertTrue(count >= memManager.getSpillBatchRowCount());
+    assertTrue(count/2 <= memManager.getSpillBatchRowCount());
+    int spillSize = memManager.getSpillBatchRowCount() * rowWidth;
+    assertTrue(spillSize <= memManager.getSpillBatchSize());
+    assertTrue(spillSize >= memManager.getSpillBatchSize()/2);
+    assertEquals(memoryLimit - memManager.getSpillBatchSize(), memManager.getBufferMemoryLimit());
+
+    // Merge sizes will also be rounded, within reason.
+
+    count = sortConfig.mergeBatchSize() / rowWidth;
+    assertTrue(count >= memManager.getMergeBatchRowCount());
+    assertTrue(count/2 <= memManager.getMergeBatchRowCount());
+    int mergeSize = memManager.getMergeBatchRowCount() * rowWidth;
+    assertTrue(mergeSize <= memManager.getMergeBatchSize());
+    assertTrue(mergeSize >= memManager.getMergeBatchSize()/2);
+    assertEquals(memoryLimit - memManager.getMergeBatchSize(), memManager.getMergeMemoryLimit());
+  }
+
+  @Test
+  public void testSmallRows() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 100 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Zero-length row, round to 10
+
+    int rowWidth = 0;
+    int rowCount = 10000;
+    int batchSize = rowCount * 2;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertEquals(10, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Truncate spill, merge batch row count
+
+    assertEquals(Character.MAX_VALUE, memManager.getSpillBatchRowCount());
+    assertEquals(Character.MAX_VALUE, memManager.getMergeBatchRowCount());
+
+    // But leave batch sizes at their defaults
+
+    assertEquals(sortConfig.spillBatchSize(), memManager.getPreferredSpillBatchSize());
+    assertEquals(sortConfig.mergeBatchSize(), memManager.getPreferredMergeBatchSize());
+
+    // Small, but non-zero, row
+
+    rowWidth = 20;
+    rowCount = 10000;
+    batchSize = rowWidth * rowCount;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+
+    // Truncate spill, merge batch row count
+
+    assertEquals(Character.MAX_VALUE, memManager.getSpillBatchRowCount());
+    assertEquals(Character.MAX_VALUE, memManager.getMergeBatchRowCount());
+
+    // But leave batch sizes at their defaults
+
+    assertEquals(sortConfig.spillBatchSize(), memManager.getPreferredSpillBatchSize());
+    assertEquals(sortConfig.mergeBatchSize(), memManager.getPreferredMergeBatchSize());
+  }
+
+  @Test
+  public void testLowMemory() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 10 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Tight squeeze, but can be made to work.
+    // Input batches are a quarter of memory.
+
+    int rowWidth = 1000;
+    int rowCount = (int) (memoryLimit / 4 / rowWidth);
+    int batchSize = rowCount * rowWidth;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertFalse(memManager.mayOverflow());
+
+    // Spill, merge batches should be constrained
+
+    int spillBatchSize = memManager.getSpillBatchSize();
+    assertTrue(spillBatchSize < memManager.getPreferredSpillBatchSize());
+    assertTrue(spillBatchSize >= rowWidth);
+    assertTrue(spillBatchSize <= memoryLimit / 3);
+    assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
+    assertTrue(spillBatchSize / rowWidth >= memManager.getSpillBatchRowCount());
+
+    int mergeBatchSize = memManager.getMergeBatchSize();
+    assertTrue(mergeBatchSize < memManager.getPreferredMergeBatchSize());
+    assertTrue(mergeBatchSize >= rowWidth);
+    assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
+    assertTrue(mergeBatchSize / rowWidth >= memManager.getMergeBatchRowCount());
+
+    // Should spill after just two batches
+
+    assertFalse(memManager.isSpillNeeded(0, batchSize));
+    assertFalse(memManager.isSpillNeeded(batchSize, batchSize));
+    assertTrue(memManager.isSpillNeeded(2 * batchSize, batchSize));
+
+    // Tighter squeeze, but can be made to work.
+    // Input batches are 3/8 of memory; two fill 3/4,
+    // but small spill and merge batches allow progress.
+
+    rowWidth = 1000;
+    rowCount = (int) (memoryLimit * 3 / 8 / rowWidth);
+    batchSize = rowCount * rowWidth;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertFalse(memManager.mayOverflow());
+
+    // Spill, merge batches should be constrained
+
+    spillBatchSize = memManager.getSpillBatchSize();
+    assertTrue(spillBatchSize < memManager.getPreferredSpillBatchSize());
+    assertTrue(spillBatchSize >= rowWidth);
+    assertTrue(spillBatchSize <= memoryLimit / 3);
+    assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
+    assertTrue(memManager.getSpillBatchRowCount() > 1);
+    assertTrue(spillBatchSize / rowWidth >= memManager.getSpillBatchRowCount());
+
+    mergeBatchSize = memManager.getMergeBatchSize();
+    assertTrue(mergeBatchSize < memManager.getPreferredMergeBatchSize());
+    assertTrue(mergeBatchSize >= rowWidth);
+    assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
+    assertTrue(memManager.getMergeBatchRowCount() > 1);
+    assertTrue(mergeBatchSize / rowWidth >= memManager.getMergeBatchRowCount());
+  }
+
+  @Test
+  public void testExtremeLowMemory() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 10 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Jumbo row size, works with one row per batch. Minimum is to have two
+    // input rows and a spill row, or two spill rows and a merge row.
+    // Have to back off the exact size a bit to allow for internal fragmentation
+    // in the merge and output batches.
+
+    int rowWidth = (int) (memoryLimit / 3 * 0.75);
+    int rowCount = 1;
+    int batchSize = rowWidth;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertEquals(rowWidth, memManager.getRowWidth());
+    assertEquals(batchSize, memManager.getInputBatchSize());
+    assertFalse(memManager.mayOverflow());
+
+    int spillBatchSize = memManager.getSpillBatchSize();
+    assertTrue(spillBatchSize >= rowWidth);
+    assertTrue(spillBatchSize <= memoryLimit / 3);
+    assertTrue(spillBatchSize + 2 * batchSize <= memoryLimit);
+    assertEquals(1, memManager.getSpillBatchRowCount());
+
+    int mergeBatchSize = memManager.getMergeBatchSize();
+    assertTrue(mergeBatchSize >= rowWidth);
+    assertTrue(mergeBatchSize + 2 * spillBatchSize <= memoryLimit);
+    assertEquals(1, memManager.getMergeBatchRowCount());
+
+    // Should spill after just two rows
+
+    assertFalse(memManager.isSpillNeeded(0, batchSize));
+    assertFalse(memManager.isSpillNeeded(batchSize, batchSize));
+    assertTrue(memManager.isSpillNeeded(2 * batchSize, batchSize));
+
+    // In trouble now, can't fit even three rows.
+
+    rowWidth = (int) (memoryLimit / 2);
+    rowCount = 1;
+    batchSize = rowWidth;
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    assertTrue(memManager.mayOverflow());
+  }
+
+  @Test
+  public void testConfigConstraints() {
+    int memConstaint = 40 * ONE_MEG;
+    int batchSizeConstaint = ONE_MEG / 2;
+    int mergeSizeConstaint = ONE_MEG;
+    DrillConfig drillConfig = new ConfigBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MAX_MEMORY, memConstaint)
+        .put(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE, batchSizeConstaint)
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE, mergeSizeConstaint)
+        .build();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 50 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    assertEquals(batchSizeConstaint, memManager.getPreferredSpillBatchSize());
+    assertEquals(mergeSizeConstaint, memManager.getPreferredMergeBatchSize());
+    assertEquals(memConstaint, memManager.getMemoryLimit());
+
+    int rowWidth = 300;
+    int rowCount = 10000;
+    int batchSize = rowWidth * rowCount * 2;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    verifyCalcs(sortConfig, memConstaint, memManager, batchSize, rowWidth, rowCount);
+  }
+
+  @Test
+  public void testMemoryDynamics() {
+    DrillConfig drillConfig = DrillConfig.create();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    long memoryLimit = 50 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    int rowWidth = 300;
+    int rowCount = 10000;
+    int batchSize = rowWidth * rowCount * 2;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+
+    int spillBatchSize = memManager.getSpillBatchSize();
+
+    // Test various memory fill levels
+
+    assertFalse(memManager.isSpillNeeded(0, batchSize));
+    assertFalse(memManager.isSpillNeeded(2 * batchSize, batchSize));
+    assertTrue(memManager.isSpillNeeded(memoryLimit - spillBatchSize + 1, batchSize));
+
+    // Similar, but for an in-memory merge
+
+    assertTrue(memManager.hasMemoryMergeCapacity(memoryLimit - ONE_MEG, ONE_MEG - 1));
+    assertTrue(memManager.hasMemoryMergeCapacity(memoryLimit - ONE_MEG, ONE_MEG));
+    assertFalse(memManager.hasMemoryMergeCapacity(memoryLimit - ONE_MEG, ONE_MEG + 1));
+  }
+
+  @Test
+  public void testMergeCalcs() {
+
+    // No artificial merge limit
+
+    int mergeLimitConstraint = 100;
+    DrillConfig drillConfig = new ConfigBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, mergeLimitConstraint)
+        .build();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    // Allow four spill batches, 8 MB each, plus one output of 16
+    long memoryLimit = 50 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Prime the estimates
+
+    int rowWidth = 300;
+    int rowCount = 10000;
+    int batchSize = rowWidth * rowCount * 2;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+    int spillBatchSize = memManager.getSpillBatchSize();
+    int mergeBatchSize = memManager.getMergeBatchSize();
+
+    // One in-mem batch, no merging.
+
+    long allocMemory = memoryLimit - mergeBatchSize;
+    MergeTask task = memManager.consolidateBatches(allocMemory, 1, 0);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // Many in-mem batches, just enough to merge
+
+    allocMemory = memoryLimit - mergeBatchSize;
+    int memBatches = (int) (allocMemory / batchSize);
+    allocMemory = memBatches * batchSize;
+    task = memManager.consolidateBatches(allocMemory, memBatches, 0);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // Spills if no room for spill and in-memory batches
+
+    task = memManager.consolidateBatches(allocMemory, memBatches, 1);
+    assertEquals(MergeAction.SPILL, task.action);
+
+    // One more in-mem batch: now needs to spill
+
+    memBatches++;
+    allocMemory = memBatches * batchSize;
+    task = memManager.consolidateBatches(allocMemory, memBatches, 0);
+    assertEquals(MergeAction.SPILL, task.action);
+
+    // No spill for various in-mem/spill run combinations
+
+    allocMemory = memoryLimit - spillBatchSize - mergeBatchSize;
+    memBatches = (int) (allocMemory / batchSize);
+    allocMemory = memBatches * batchSize;
+    task = memManager.consolidateBatches(allocMemory, memBatches, 1);
+    assertEquals(MergeAction.NONE, task.action);
+
+    allocMemory = memoryLimit - 2 * spillBatchSize - mergeBatchSize;
+    memBatches = (int) (allocMemory / batchSize);
+    allocMemory = memBatches * batchSize;
+    task = memManager.consolidateBatches(allocMemory, memBatches, 2);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // No spill if no in-memory, only spill, and spill fits
+
+    long freeMem = memoryLimit - mergeBatchSize;
+    int spillBatches = (int) (freeMem / spillBatchSize);
+    task = memManager.consolidateBatches(0, 0, spillBatches);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // One more and must merge
+
+    task = memManager.consolidateBatches(0, 0, spillBatches + 1);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(2, task.count);
+
+    // Two more and will merge more
+
+    task = memManager.consolidateBatches(0, 0, spillBatches + 2);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(3, task.count);
+  }
+
+  @Test
+  public void testMergeLimit() {
+    // Constrain merge width
+    int mergeLimitConstraint = 5;
+    DrillConfig drillConfig = new ConfigBuilder()
+        .put(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT, mergeLimitConstraint)
+        .build();
+    SortConfig sortConfig = new SortConfig(drillConfig);
+    // Plenty of memory, memory will not be a limit
+    long memoryLimit = 400 * ONE_MEG;
+    SortMemoryManager memManager = new SortMemoryManager(sortConfig, memoryLimit);
+
+    // Prime the estimates
+
+    int rowWidth = 300;
+    int rowCount = 10000;
+    int batchSize = rowWidth * rowCount * 2;
+
+    memManager.updateEstimates(batchSize, rowWidth, rowCount);
+
+    // Pretend merge limit runs, additional in-memory batches
+
+    int memBatchCount = 10;
+    int spillRunCount = mergeLimitConstraint;
+    long allocMemory = batchSize * memBatchCount;
+    MergeTask task = memManager.consolidateBatches(allocMemory, memBatchCount, spillRunCount);
+    assertEquals(MergeAction.NONE, task.action);
+
+    // One more run than can merge in one go. But, we have plenty of
+    // memory to merge and hold the in-memory batches. So, just merge.
+
+    task = memManager.consolidateBatches(allocMemory, memBatchCount, spillRunCount + 1);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(2, task.count);
+
+    // One more runs than can merge in one go, intermediate merge
+
+    task = memManager.consolidateBatches(0, 0, spillRunCount + 1);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(2, task.count);
+
+    // Two more spill runs, merge three
+
+    task = memManager.consolidateBatches(0, 0, spillRunCount + 2);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(3, task.count);
+
+    // Way more than can merge, limit to the constraint
+
+    task = memManager.consolidateBatches(0, 0, spillRunCount * 3);
+    assertEquals(MergeAction.MERGE, task.action);
+    assertEquals(mergeLimitConstraint, task.count);
+  }
+
+  @Test
+  public void testMetrics() {
+    OperatorFixture.MockStats stats = new OperatorFixture.MockStats();
+    SortMetrics metrics = new SortMetrics(stats);
+
+    // Input stats
+
+    metrics.updateInputMetrics(100, 10_000);
+    assertEquals(1, metrics.getInputBatchCount());
+    assertEquals(100, metrics.getInputRowCount());
+    assertEquals(10_000, metrics.getInputBytes());
+
+    metrics.updateInputMetrics(200, 20_000);
+    assertEquals(2, metrics.getInputBatchCount());
+    assertEquals(300, metrics.getInputRowCount());
+    assertEquals(30_000, metrics.getInputBytes());
+
+    // Buffer memory
+
+    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+
+    metrics.updateMemory(1_000_000);
+    assertEquals(1_000_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+
+    metrics.updateMemory(2_000_000);
+    assertEquals(1_000_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+
+    metrics.updateMemory(100_000);
+    assertEquals(100_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+
+    // Peak batches
+
+    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+
+    metrics.updatePeakBatches(10);
+    assertEquals(10D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+
+    metrics.updatePeakBatches(1);
+    assertEquals(10D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+
+    metrics.updatePeakBatches(20);
+    assertEquals(20D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+
+    // Merge count
+
+    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+
+    metrics.incrMergeCount();
+    assertEquals(1D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+
+    metrics.incrMergeCount();
+    assertEquals(2D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+
+    // Spill count
+
+    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+
+    metrics.incrSpillCount();
+    assertEquals(1D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+
+    metrics.incrSpillCount();
+    assertEquals(2D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+
+    // Write bytes
+
+    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.SPILL_MB), 0.01);
+
+    metrics.updateWriteBytes(17 * ONE_MEG + ONE_MEG * 3 / 4);
+    assertEquals(17.75D, stats.getStat(ExternalSortBatch.Metric.SPILL_MB), 0.001);
+  }
+}


[3/5] drill git commit: DRILL-5325: Unit tests for the managed sort

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
index 76b178c..7a460f5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierTemplate.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl.xsort.managed;
 
-import io.netty.buffer.DrillBuf;
-
 import java.io.IOException;
 import java.util.List;
 
@@ -26,11 +24,11 @@ import javax.inject.Named;
 
 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;
-import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.AllocationHelper;
+
+import io.netty.buffer.DrillBuf;
 
 public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierTemplate.class);
@@ -43,7 +41,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
   private int queueSize = 0;
 
   @Override
-  public void setup(FragmentContext context, BufferAllocator allocator, VectorAccessible hyperBatch, List<BatchGroup> batchGroups,
+  public void setup(BufferAllocator allocator, VectorAccessible hyperBatch, List<BatchGroup> batchGroups,
                     VectorAccessible outgoing) throws SchemaChangeException {
     this.hyperBatch = hyperBatch;
     this.batchGroups = batchGroups;
@@ -53,7 +51,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     @SuppressWarnings("resource")
     final DrillBuf drillBuf = allocator.buffer(4 * size);
     vector4 = new SelectionVector4(drillBuf, size, Character.MAX_VALUE);
-    doSetup(context, hyperBatch, outgoing);
+    doSetup(hyperBatch, outgoing);
 
     queueSize = 0;
     for (int i = 0; i < size; i++) {
@@ -68,7 +66,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
 
   @Override
   public int next(int targetRecordCount) {
-    allocateVectors(targetRecordCount);
+    VectorAccessibleUtilities.allocateVectors(outgoing, targetRecordCount);
     for (int outgoingIndex = 0; outgoingIndex < targetRecordCount; outgoingIndex++) {
       if (queueSize == 0) {
         return 0;
@@ -76,7 +74,11 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
       int compoundIndex = vector4.get(0);
       int batch = compoundIndex >>> 16;
       assert batch < batchGroups.size() : String.format("batch: %d batchGroups: %d", batch, batchGroups.size());
-      doCopy(compoundIndex, outgoingIndex);
+      try {
+        doCopy(compoundIndex, outgoingIndex);
+      } catch (SchemaChangeException e) {
+        throw new IllegalStateException(e);
+      }
       int nextIndex = batchGroups.get(batch).getNextIndex();
       if (nextIndex < 0) {
         vector4.set(0, vector4.get(--queueSize));
@@ -84,37 +86,28 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
         vector4.set(0, batch, nextIndex);
       }
       if (queueSize == 0) {
-        setValueCount(++outgoingIndex);
+        VectorAccessibleUtilities.setValueCount(outgoing, ++outgoingIndex);
         return outgoingIndex;
       }
-      siftDown();
+      try {
+        siftDown();
+      } catch (SchemaChangeException e) {
+        throw new IllegalStateException(e);
+      }
     }
-    setValueCount(targetRecordCount);
+    VectorAccessibleUtilities.setValueCount(outgoing, targetRecordCount);
     return targetRecordCount;
   }
 
-  private void setValueCount(int count) {
-    for (VectorWrapper<?> w: outgoing) {
-      w.getValueVector().getMutator().setValueCount(count);
-    }
-  }
-
   @Override
   public void close() throws IOException {
     vector4.clear();
-    for (final VectorWrapper<?> w: outgoing) {
-      w.getValueVector().clear();
-    }
-    for (final VectorWrapper<?> w : hyperBatch) {
-      w.clear();
-    }
-
-    for (BatchGroup batchGroup : batchGroups) {
-      batchGroup.close();
-    }
+    VectorAccessibleUtilities.clear(outgoing);
+    VectorAccessibleUtilities.clear(hyperBatch);
+    BatchGroup.closeAll(batchGroups);
   }
 
-  private void siftUp() {
+  private void siftUp() throws SchemaChangeException {
     int p = queueSize;
     while (p > 0) {
       if (compare(p, (p - 1) / 2) < 0) {
@@ -126,13 +119,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     }
   }
 
-  private void allocateVectors(int targetRecordCount) {
-    for (VectorWrapper<?> w: outgoing) {
-      AllocationHelper.allocateNew(w.getValueVector(), targetRecordCount);
-    }
-  }
-
-  private void siftDown() {
+  private void siftDown() throws SchemaChangeException {
     int p = 0;
     int next;
     while (p * 2 + 1 < queueSize) { // While the current node has at least one child
@@ -160,14 +147,19 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     vector4.set(sv1, tmp);
   }
 
-  public int compare(int leftIndex, int rightIndex) {
+  public int compare(int leftIndex, int rightIndex) throws SchemaChangeException {
     int sv1 = vector4.get(leftIndex);
     int sv2 = vector4.get(rightIndex);
     return doEval(sv1, sv2);
   }
 
-  public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorAccessible incoming, @Named("outgoing") VectorAccessible outgoing);
-  public abstract int doEval(@Named("leftIndex") int leftIndex, @Named("rightIndex") int rightIndex);
-  public abstract void doCopy(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex);
-
+  public abstract void doSetup(@Named("incoming") VectorAccessible incoming,
+                               @Named("outgoing") VectorAccessible outgoing)
+                       throws SchemaChangeException;
+  public abstract int doEval(@Named("leftIndex") int leftIndex,
+                             @Named("rightIndex") int rightIndex)
+                      throws SchemaChangeException;
+  public abstract void doCopy(@Named("inIndex") int inIndex,
+                              @Named("outIndex") int outIndex)
+                       throws SchemaChangeException;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
new file mode 100644
index 0000000..6b71782
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
@@ -0,0 +1,341 @@
+/*
+ * 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 java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.compile.sig.GeneratorMapping;
+import org.apache.drill.exec.compile.sig.MappingSet;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.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.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.CopyUtil;
+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 PriorityQueueCopierWrapper extends BaseSortWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueCopierWrapper.class);
+
+  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);
+
+  /**
+   * 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;
+
+  public PriorityQueueCopierWrapper(OperExecContext opContext) {
+    super(opContext);
+  }
+
+  public PriorityQueueCopier getCopier(VectorAccessible batch) {
+    if (copier == null) {
+      copier = newCopier(batch);
+    }
+    return copier;
+  }
+
+  private PriorityQueueCopier newCopier(VectorAccessible batch) {
+    // Generate the copier code and obtain the resulting class
+
+    CodeGenerator<PriorityQueueCopier> cg = CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptionSet());
+    ClassGenerator<PriorityQueueCopier> g = cg.getRoot();
+    cg.plainJavaCapable(true);
+    // Uncomment out this line to debug the generated code.
+//    cg.saveCodeForDebugging(true);
+
+    generateComparisons(g, batch, logger);
+
+    g.setMappingSet(COPIER_MAPPING_SET);
+    CopyUtil.generateCopies(g, batch, true);
+    g.setMappingSet(MAIN_MAPPING);
+    return getInstance(cg, logger);
+  }
+
+  /**
+   * 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 BatchMerger startMerge(BatchSchema schema, List<? extends BatchGroup> 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 do the actual writes. If the copier has not yet been
+   * created, generate 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
+   */
+
+  @SuppressWarnings("unchecked")
+  private void createCopier(VectorAccessible batch, List<? extends BatchGroup> batchGroupList, VectorContainer outputContainer) {
+    copier = getCopier(batch);
+
+    // Initialize the value vectors for the output container
+
+    for (VectorWrapper<?> i : batch) {
+      @SuppressWarnings("resource")
+      ValueVector v = TypeHelper.getNewVector(i.getField(), context.getAllocator());
+      outputContainer.add(v);
+    }
+    try {
+      copier.setup(context.getAllocator(), batch, (List<BatchGroup>) batchGroupList, outputContainer);
+    } catch (SchemaChangeException e) {
+      throw UserException.unsupportedError(e)
+            .message("Unexpected schema change - likely code error.")
+            .build(logger);
+    }
+  }
+
+  public BufferAllocator getAllocator() { return context.getAllocator(); }
+
+  public void close() {
+    if (copier == null) {
+      return; }
+    try {
+      copier.close();
+      copier = null;
+    } catch (IOException e) {
+      throw UserException.dataWriteError(e)
+            .message("Failure while flushing spilled data")
+            .build(logger);
+    }
+  }
+
+  /**
+   * We've gathered a set of batches, each of which has been sorted. The batches
+   * may have passed through a filter and thus may have "holes" where rows have
+   * been filtered out. We will spill records in blocks of targetRecordCount.
+   * To prepare, copy that many records into an outputContainer as a set of
+   * contiguous values in new vectors. The result is a single batch with
+   * vectors that combine a collection of input batches up to the
+   * given threshold.
+   * <p>
+   * Input. Here the top line is a selection vector of indexes.
+   * The second line is a set of batch groups (separated by underscores)
+   * with letters indicating individual records:<pre>
+   * [3 7 4 8 0 6 1] [5 3 6 8 2 0]
+   * [eh_ad_ibf]     [r_qm_kn_p]</pre>
+   * <p>
+   * Output, assuming blocks of 5 records. The brackets represent
+   * batches, the line represents the set of batches copied to the
+   * spill file.<pre>
+   * [abcde] [fhikm] [npqr]</pre>
+   * <p>
+   * The copying operation does a merge as well: copying
+   * values from the sources in ordered fashion. Consider a different example,
+   * we want to merge two input batches to produce a single output batch:
+   * <pre>
+   * Input:  [aceg] [bdfh]
+   * Output: [abcdefgh]</pre>
+   * <p>
+   * In the above, the input consists of two sorted batches. (In reality,
+   * the input batches have an associated selection vector, but that is omitted
+   * here and just the sorted values shown.) The output is a single batch
+   * with the merged records (indicated by letters) from the two input batches.
+   * <p>
+   * Here we bind the copier to the batchGroupList of sorted, buffered batches
+   * to be merged. We bind the copier output to outputContainer: the copier will write its
+   * merged "batches" of records to that container.
+   * <p>
+   * Calls to the {@link #next()} method sequentially return merged batches
+   * of the desired row count.
+    */
+
+  public static class BatchMerger implements SortResults, AutoCloseable {
+
+    private PriorityQueueCopierWrapper holder;
+    private VectorContainer hyperBatch;
+    private VectorContainer outputContainer;
+    private int targetRecordCount;
+    private int copyCount;
+    private int batchCount;
+    private long estBatchSize;
+
+    /**
+     * Creates a merger with an temporary output container.
+     *
+     * @param holder the copier that does the work
+     * @param schema schema for the input and output batches
+     * @param batchGroupList the input batches
+     * @param targetRecordCount number of records for each output batch
+     */
+    private BatchMerger(PriorityQueueCopierWrapper holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
+                        int targetRecordCount) {
+      this(holder, schema, batchGroupList, new VectorContainer(), targetRecordCount);
+    }
+
+    /**
+     * Creates a merger with the specified output container
+     *
+     * @param holder the copier that does the work
+     * @param schema schema for the input and output batches
+     * @param batchGroupList the input batches
+     * @param outputContainer merges output batch into the given output container
+     * @param targetRecordCount number of records for each output batch
+     */
+    private BatchMerger(PriorityQueueCopierWrapper holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
+                        VectorContainer outputContainer, int targetRecordCount) {
+      this.holder = holder;
+      hyperBatch = constructHyperBatch(schema, batchGroupList);
+      copyCount = 0;
+      this.targetRecordCount = targetRecordCount;
+      this.outputContainer = outputContainer;
+      holder.createCopier(hyperBatch, batchGroupList, outputContainer);
+    }
+
+    /**
+     * Read the next merged batch. The batch holds the specified row count, but
+     * may be less if this is the last batch.
+     *
+     * @return the number of rows in the batch, or 0 if no more batches
+     * are available
+     */
+
+    @Override
+    public boolean next() {
+      Stopwatch w = Stopwatch.createStarted();
+      long start = holder.getAllocator().getAllocatedMemory();
+      int count = holder.copier.next(targetRecordCount);
+      copyCount += count;
+      if (count > 0) {
+        long t = w.elapsed(TimeUnit.MICROSECONDS);
+        batchCount++;
+        logger.trace("Took {} us to merge {} records", t, count);
+        long size = holder.getAllocator().getAllocatedMemory() - start;
+        estBatchSize = Math.max(estBatchSize, size);
+      } else {
+        logger.trace("copier returned 0 records");
+      }
+
+      // Identify the schema to be used in the output container. (Since
+      // all merged batches have the same schema, the schema we identify
+      // here should be the same as that which we already had.
+
+      outputContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+
+      // The copier does not set the record count in the output
+      // container, so do that here.
+
+      outputContainer.setRecordCount(count);
+
+      return count > 0;
+    }
+
+    /**
+     * Construct a vector container that holds a list of batches, each represented as an
+     * array of vectors. The entire collection of vectors has a common schema.
+     * <p>
+     * To build the collection, we go through the current schema (which has been
+     * devised to be common for all batches.) For each field in the schema, we create
+     * an array of vectors. To create the elements, we iterate over all the incoming
+     * batches and search for the vector that matches the current column.
+     * <p>
+     * Finally, we build a new schema for the combined container. That new schema must,
+     * because of the way the container was created, match the current schema.
+     *
+     * @param schema schema for the hyper batch
+     * @param batchGroupList list of batches to combine
+     * @return a container where each column is represented as an array of vectors
+     * (hence the "hyper" in the method name)
+     */
+
+    private VectorContainer constructHyperBatch(BatchSchema schema, List<? extends BatchGroup> batchGroupList) {
+      VectorContainer cont = new VectorContainer();
+      for (MaterializedField field : schema) {
+        ValueVector[] vectors = new ValueVector[batchGroupList.size()];
+        int i = 0;
+        for (BatchGroup group : batchGroupList) {
+          vectors[i++] = group.getValueAccessorById(
+              field.getValueClass(),
+              group.getValueVectorId(SchemaPath.getSimplePath(field.getPath())).getFieldIds())
+              .getValueVector();
+        }
+        cont.add(vectors);
+      }
+      cont.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);
+      return cont;
+    }
+
+    @Override
+    public void close() {
+      hyperBatch.clear();
+      holder.close();
+    }
+
+    @Override
+    public int getRecordCount() { return outputContainer.getRecordCount(); }
+
+    @Override
+    public int getBatchCount() { return batchCount; }
+
+    /**
+     * Gets the estimated batch size, in bytes. Use for estimating the memory
+     * needed to process the batches that this operator created.
+     * @return the size of the largest batch created by this operation,
+     * in bytes
+     */
+
+    public long getEstBatchSize() { return estBatchSize; }
+
+    @Override
+    public SelectionVector4 getSv4() { return null; }
+
+    @Override
+    public SelectionVector2 getSv2() { return null; }
+
+    @Override
+    public VectorContainer getContainer() { return outputContainer; }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.java
new file mode 100644
index 0000000..e47d67e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortConfig.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 org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+
+public class SortConfig {
+
+  /**
+   * Smallest allowed output batch size. The smallest output batch
+   * created even under constrained memory conditions.
+   */
+  public static final int MIN_MERGED_BATCH_SIZE = 256 * 1024;
+
+  /**
+   * In the bizarre case where the user gave us an unrealistically low
+   * spill file size, set a floor at some bare minimum size. (Note that,
+   * at this size, big queries will create a huge number of files, which
+   * is why the configuration default is one the order of hundreds of MB.)
+   */
+
+  public static final long MIN_SPILL_FILE_SIZE = 1 * 1024 * 1024;
+
+  public static final int DEFAULT_SPILL_BATCH_SIZE = 8 * 1024 * 1024;
+  public static final int MIN_SPILL_BATCH_SIZE = 256 * 1024;
+  public static final int MIN_MERGE_BATCH_SIZE = 256 * 1024;
+
+  public static final int MIN_MERGE_LIMIT = 2;
+
+  private final long maxMemory;
+
+  /**
+   * Maximum number of spilled runs that can be merged in a single pass.
+   */
+
+  private final int mergeLimit;
+
+  /**
+   * Target size of the first-generation spill files.
+   */
+  private final long spillFileSize;
+
+  private final int spillBatchSize;
+
+  private final int mergeBatchSize;
+
+  private final int bufferedBatchLimit;
+
+
+  public SortConfig(DrillConfig config) {
+
+    // Optional configured memory limit, typically used only for testing.
+
+    maxMemory = config.getBytes(ExecConstants.EXTERNAL_SORT_MAX_MEMORY);
+
+    // Optional limit on the number of spilled runs to merge in a single
+    // pass. Limits the number of open file handles. Must allow at least
+    // two batches to merge to make progress.
+
+    int limit = config.getInt(ExecConstants.EXTERNAL_SORT_MERGE_LIMIT);
+    if (limit > 0) {
+      mergeLimit = Math.max(limit, MIN_MERGE_LIMIT);
+    } else {
+      mergeLimit = Integer.MAX_VALUE;
+    }
+
+    // Limits the size of first-generation spill files.
+    // Ensure the size is reasonable.
+
+    spillFileSize = Math.max(config.getBytes(ExecConstants.EXTERNAL_SORT_SPILL_FILE_SIZE), MIN_SPILL_FILE_SIZE);
+    spillBatchSize = (int) Math.max(config.getBytes(ExecConstants.EXTERNAL_SORT_SPILL_BATCH_SIZE), MIN_SPILL_BATCH_SIZE);
+
+    // Set the target output batch size. Use the maximum size, but only if
+    // this represents less than 10% of available memory. Otherwise, use 10%
+    // of memory, but no smaller than the minimum size. In any event, an
+    // output batch can contain no fewer than a single record.
+
+    mergeBatchSize = (int) Math.max(config.getBytes(ExecConstants.EXTERNAL_SORT_MERGE_BATCH_SIZE), MIN_MERGE_BATCH_SIZE);
+
+    // Limit on in-memory batches, primarily for testing.
+
+    int value = config.getInt(ExecConstants.EXTERNAL_SORT_BATCH_LIMIT);
+    if (value == 0) {
+      bufferedBatchLimit = Integer.MAX_VALUE;
+    } else {
+      bufferedBatchLimit = Math.max(value, 2);
+    }
+    logConfig();
+  }
+
+  private void logConfig() {
+    ExternalSortBatch.logger.debug("Config: " +
+                 "spill file size = {}, spill batch size = {}, " +
+                 "merge limit = {}, merge batch size = {}",
+                  spillFileSize(), spillFileSize(),
+                  mergeLimit(), mergeBatchSize());
+  }
+
+  public long maxMemory() { return maxMemory; }
+  public int mergeLimit() { return mergeLimit; }
+  public long spillFileSize() { return spillFileSize; }
+  public int spillBatchSize() { return spillBatchSize; }
+  public int mergeBatchSize() { return mergeBatchSize; }
+  public int getBufferedBatchLimit() { return bufferedBatchLimit; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
new file mode 100644
index 0000000..6f0da3d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
@@ -0,0 +1,491 @@
+/*
+ * 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.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.impl.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.impl.xsort.MSortTemplate;
+import org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeTask;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorAccessibleUtilities;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+
+/**
+ * Implementation of the external sort which is wrapped into the Drill
+ * "next" protocol by the {@link ExternalSortBatch} class.
+ * <p>
+ * Accepts incoming batches. Sorts each and will spill to disk as needed.
+ * When all input is delivered, can either do an in-memory merge or a
+ * merge from disk. If runs spilled, may have to do one or more "consolidation"
+ * passes to reduce the number of runs to the level that will fit in memory.
+ */
+
+public class SortImpl {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSortBatch.class);
+
+  /**
+   * Iterates over the final sorted results. Implemented differently
+   * depending on whether the results are in-memory or spilled to
+   * disk.
+   */
+
+  public interface SortResults {
+    /**
+     * Container into which results are delivered. May the
+     * the original operator container, or may be a different
+     * one. This is the container that should be sent
+     * downstream. This is a fixed value for all returned
+     * results.
+     * @return
+     */
+    VectorContainer getContainer();
+    boolean next();
+    void close();
+    int getBatchCount();
+    int getRecordCount();
+    SelectionVector2 getSv2();
+    SelectionVector4 getSv4();
+  }
+
+  private final SortConfig config;
+  private final SortMetrics metrics;
+  private final SortMemoryManager memManager;
+  private VectorContainer outputBatch;
+  private OperExecContext context;
+
+  /**
+   * Memory allocator for this operator itself. Incoming batches are
+   * transferred into this allocator. Intermediate batches used during
+   * merge also reside here.
+   */
+
+  private final BufferAllocator allocator;
+
+  private final SpilledRuns spilledRuns;
+
+  private final BufferedBatches bufferedBatches;
+
+  public SortImpl(OperExecContext opContext, SortConfig sortConfig, SpilledRuns spilledRuns, VectorContainer batch) {
+    this.context = opContext;
+    outputBatch = batch;
+    this.spilledRuns = spilledRuns;
+    allocator = opContext.getAllocator();
+    config = sortConfig;
+    memManager = new SortMemoryManager(config, allocator.getLimit());
+    metrics = new SortMetrics(opContext.getStats());
+    bufferedBatches = new BufferedBatches(opContext);
+
+    // Reset the allocator to allow a 10% safety margin. This is done because
+    // the memory manager will enforce the original limit. Changing the hard
+    // limit will reduce the probability that random chance causes the allocator
+    // to kill the query because of a small, spurious over-allocation.
+
+    allocator.setLimit((long)(allocator.getLimit() * 1.10));
+  }
+
+  public void setSchema(BatchSchema schema) {
+    bufferedBatches.setSchema(schema);
+    spilledRuns.setSchema(schema);
+  }
+
+  public boolean forceSpill() {
+    if (bufferedBatches.size() < 2) {
+      return false;
+    }
+    spillFromMemory();
+    return true;
+  }
+
+  /**
+   * Process the converted incoming batch by adding it to the in-memory store
+   * of data, or spilling data to disk when necessary.
+   * @param incoming
+   */
+
+  public void addBatch(VectorAccessible incoming) {
+
+    // Skip empty batches (such as the first one.)
+
+    if (incoming.getRecordCount() == 0) {
+      VectorAccessibleUtilities.clear(incoming);
+      return;
+    }
+
+    // Determine actual sizes of the incoming batch before taking
+    // ownership. Allows us to figure out if we need to spill first,
+    // to avoid overflowing memory simply due to ownership transfer.
+
+    RecordBatchSizer sizer = analyzeIncomingBatch(incoming);
+
+    // The heart of the external sort operator: spill to disk when
+    // the in-memory generation exceeds the allowed memory limit.
+    // Preemptively spill BEFORE accepting the new batch into our memory
+    // pool. The allocator will throw an OOM exception if we accept the
+    // batch when we are near the limit - despite the fact that the batch
+    // is already in memory and no new memory is allocated during the transfer.
+
+    if ( isSpillNeeded(sizer.actualSize())) {
+      spillFromMemory();
+    }
+
+    // Sanity check. We should now be below the buffer memory maximum.
+
+    long startMem = allocator.getAllocatedMemory();
+    bufferedBatches.add(incoming, sizer.netSize());
+
+    // Compute batch size, including allocation of an sv2.
+
+    long endMem = allocator.getAllocatedMemory();
+    long batchSize = endMem - startMem;
+
+    // Update the minimum buffer space metric.
+
+    metrics.updateInputMetrics(sizer.rowCount(), sizer.actualSize());
+    metrics.updateMemory(memManager.freeMemory(endMem));
+    metrics.updatePeakBatches(bufferedBatches.size());
+
+    // Update the size based on the actual record count, not
+    // the effective count as given by the selection vector
+    // (which may exclude some records due to filtering.)
+
+    validateBatchSize(sizer.actualSize(), batchSize);
+    memManager.updateEstimates((int) batchSize, sizer.netRowWidth(), sizer.rowCount());
+  }
+
+  /**
+   * Scan the vectors in the incoming batch to determine batch size.
+   *
+   * @return an analysis of the incoming batch
+   */
+
+  private RecordBatchSizer analyzeIncomingBatch(VectorAccessible incoming) {
+    RecordBatchSizer sizer = new RecordBatchSizer(incoming);
+    sizer.applySv2();
+    if (metrics.getInputBatchCount() == 0) {
+      logger.debug("{}", sizer.toString());
+    }
+    return sizer;
+  }
+
+  /**
+   * Determine if spill is needed before receiving the new record batch.
+   * Spilling is driven purely by memory availability (and an optional
+   * batch limit for testing.)
+   *
+   * @return true if spilling is needed, false otherwise
+   */
+
+  private boolean isSpillNeeded(int incomingSize) {
+
+    // Can't spill if less than two batches else the merge
+    // can't make progress.
+
+    if (bufferedBatches.size() < 2) {
+      return false; }
+
+    if (bufferedBatches.size() >= config.getBufferedBatchLimit()) {
+      return true; }
+    return memManager.isSpillNeeded(allocator.getAllocatedMemory(), incomingSize);
+  }
+
+  private void validateBatchSize(long actualBatchSize, long memoryDelta) {
+    if (actualBatchSize != memoryDelta) {
+      ExternalSortBatch.logger.debug("Memory delta: {}, actual batch size: {}, Diff: {}",
+                   memoryDelta, actualBatchSize, memoryDelta - actualBatchSize);
+    }
+  }
+
+  /**
+   * This operator has accumulated a set of sorted incoming record batches.
+   * We wish to spill some of them to disk. To do this, a "copier"
+   * merges the target batches to produce a stream of new (merged) batches
+   * which are then written to disk.
+   * <p>
+   * This method spills only half the accumulated batches
+   * minimizing unnecessary disk writes. The exact count must lie between
+   * the minimum and maximum spill counts.
+   */
+
+  private void spillFromMemory() {
+    int startCount = bufferedBatches.size();
+    List<BatchGroup> batchesToSpill = bufferedBatches.prepareSpill(config.spillFileSize());
+
+    // Do the actual spill.
+
+    logger.trace("Spilling {} of {} batches, memory = {}",
+        batchesToSpill.size(), startCount,
+        allocator.getAllocatedMemory());
+    int spillBatchRowCount = memManager.getSpillBatchRowCount();
+    spilledRuns.mergeAndSpill(batchesToSpill, spillBatchRowCount);
+    metrics.incrSpillCount();
+  }
+
+  public SortMetrics getMetrics() { return metrics; }
+
+  public static class EmptyResults implements SortResults {
+
+    private final VectorContainer dest;
+
+    public EmptyResults(VectorContainer dest) {
+      this.dest = dest;
+    }
+
+    @Override
+    public boolean next() { return false; }
+
+    @Override
+    public void close() { }
+
+    @Override
+    public int getBatchCount() { return 0; }
+
+    @Override
+    public int getRecordCount() { return 0; }
+
+    @Override
+    public SelectionVector4 getSv4() { return null; }
+
+    @Override
+    public SelectionVector2 getSv2() { return null; }
+
+    @Override
+    public VectorContainer getContainer() { return dest; }
+  }
+
+  public SortResults startMerge() {
+    if (metrics.getInputRowCount() == 0) {
+      return new EmptyResults(outputBatch);
+    }
+
+    logger.debug("Completed load phase: read {} batches, spilled {} times, total input bytes: {}",
+        metrics.getInputBatchCount(), spilledRuns.size(),
+        metrics.getInputBytes());
+
+    // Do the merge of the loaded batches. The merge can be done entirely in memory if
+    // the results fit; else we have to do a disk-based merge of
+    // pre-sorted spilled batches.
+
+    boolean optimizeOn = true; // Debug only
+    if (optimizeOn && metrics.getInputBatchCount() == 1) {
+      return singleBatchResult();
+    } else if (canUseMemoryMerge()) {
+      return mergeInMemory();
+    } else {
+      return mergeSpilledRuns();
+    }
+  }
+
+  /**
+   * Return results for a single input batch. No merge is needed;
+   * the original (sorted) input batch is simply passed as the result.
+   * Note that this version requires replacing the operator output
+   * container with the batch container. (Vector ownership transfer
+   * was already done when accepting the input batch.)
+   */
+
+  public static class SingleBatchResults implements SortResults {
+
+    private boolean done;
+    private final BatchGroup.InputBatch batch;
+
+    public SingleBatchResults(BatchGroup.InputBatch batch) {
+      this.batch = batch;
+    }
+
+    @Override
+    public boolean next() {
+      if (done) {
+        return false;
+      }
+      done = true;
+      return true;
+    }
+
+    @Override
+    public void close() {
+      try {
+        batch.close();
+      } catch (IOException e) {
+        // Should never occur for an input batch
+        throw new IllegalStateException(e);
+      }
+    }
+
+    @Override
+    public int getBatchCount() { return 1; }
+
+    @Override
+    public int getRecordCount() { return batch.getRecordCount(); }
+
+    @Override
+    public SelectionVector4 getSv4() { return null; }
+
+    @Override
+    public SelectionVector2 getSv2() { return batch.getSv2(); }
+
+    @Override
+    public VectorContainer getContainer() {return batch.getContainer(); }
+  }
+
+  /**
+   * Input consists of a single batch. Just return that batch as
+   * the output.
+   * @return results iterator over the single input batch
+   */
+
+  private SortResults singleBatchResult() {
+    List<InputBatch> batches = bufferedBatches.removeAll();
+    return new SingleBatchResults(batches.get(0));
+  }
+
+  /**
+   * All data has been read from the upstream batch. Determine if we
+   * can use a fast in-memory sort, or must use a merge (which typically,
+   * but not always, involves spilled batches.)
+   *
+   * @return whether sufficient resources exist to do an in-memory sort
+   * if all batches are still in memory
+   */
+
+  private boolean canUseMemoryMerge() {
+    if (spilledRuns.hasSpilled()) {
+      return false; }
+
+    // Do we have enough memory for MSorter (the in-memory sorter)?
+
+    if (! memManager.hasMemoryMergeCapacity(allocator.getAllocatedMemory(), MSortTemplate.memoryNeeded(metrics.getInputRowCount()))) {
+      return false; }
+
+    // Make sure we don't exceed the maximum number of batches SV4 can address.
+
+    if (bufferedBatches.size() > Character.MAX_VALUE) {
+      return false; }
+
+    // We can do an in-memory merge.
+
+    return true;
+  }
+
+  /**
+   * Perform an in-memory sort of the buffered batches. Obviously can
+   * be used only for the non-spilling case.
+   *
+   * @return DONE if no rows, OK_NEW_SCHEMA if at least one row
+   */
+
+  private SortResults mergeInMemory() {
+    logger.debug("Starting in-memory sort. Batches = {}, Records = {}, Memory = {}",
+                 bufferedBatches.size(), metrics.getInputRowCount(),
+                 allocator.getAllocatedMemory());
+
+    // Note the difference between how we handle batches here and in the spill/merge
+    // case. In the spill/merge case, this class decides on the batch size to send
+    // downstream. However, in the in-memory case, we must pass along all batches
+    // in a single SV4. Attempts to do paging will result in errors. In the memory
+    // merge case, the downstream Selection Vector Remover will split the one
+    // big SV4 into multiple smaller batches to send further downstream.
+
+    // If the sort fails or is empty, clean up here. Otherwise, cleanup is done
+    // by closing the resultsIterator after all results are returned downstream.
+
+    MergeSortWrapper memoryMerge = new MergeSortWrapper(context, outputBatch);
+    try {
+      memoryMerge.merge(bufferedBatches.removeAll());
+    } catch (Throwable t) {
+      memoryMerge.close();
+      throw t;
+    }
+    logger.debug("Completed in-memory sort. Memory = {}",
+                 allocator.getAllocatedMemory());
+    return memoryMerge;
+  }
+
+  /**
+   * Perform merging of (typically spilled) batches. First consolidates batches
+   * as needed, then performs a final merge that is read one batch at a time
+   * to deliver batches to the downstream operator.
+   *
+   * @return an iterator over the merged batches
+   */
+
+  private SortResults mergeSpilledRuns() {
+    logger.debug("Starting consolidate phase. Batches = {}, Records = {}, Memory = {}, In-memory batches {}, spilled runs {}",
+                 metrics.getInputBatchCount(), metrics.getInputRowCount(),
+                 allocator.getAllocatedMemory(),
+                 bufferedBatches.size(), spilledRuns.size());
+
+    // Consolidate batches to a number that can be merged in
+    // a single last pass.
+
+    loop:
+    for (;;) {
+      MergeTask task = memManager.consolidateBatches(
+          allocator.getAllocatedMemory(),
+          bufferedBatches.size(),
+          spilledRuns.size());
+      switch (task.action) {
+      case SPILL:
+        spillFromMemory();
+        break;
+      case MERGE:
+        mergeRuns(task.count);
+        break;
+      case NONE:
+        break loop;
+      default:
+        throw new IllegalStateException("Unexpected action: " + task.action);
+      }
+    }
+
+    int mergeRowCount = memManager.getMergeBatchRowCount();
+    return spilledRuns.finalMerge(bufferedBatches.removeAll(), outputBatch, mergeRowCount);
+  }
+
+  private void mergeRuns(int targetCount) {
+    long mergeMemoryPool = memManager.getMergeMemoryLimit();
+    int spillBatchRowCount = memManager.getSpillBatchRowCount();
+    spilledRuns.mergeRuns(targetCount, mergeMemoryPool, spillBatchRowCount);
+    metrics.incrMergeCount();
+  }
+
+  public void close() {
+    metrics.updateWriteBytes(spilledRuns.getWriteBytes());
+    RuntimeException ex = null;
+    try {
+      spilledRuns.close();
+    } catch (RuntimeException e) {
+      ex = e;
+    }
+    try {
+      bufferedBatches.close();
+    } catch (RuntimeException e) {
+      ex = ex == null ? e : ex;
+    }
+    if (ex != null) {
+      throw ex;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
new file mode 100644
index 0000000..213720f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMemoryManager.java
@@ -0,0 +1,513 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+
+public class SortMemoryManager {
+
+  /**
+   * Maximum memory this operator may use. Usually comes from the
+   * operator definition, but may be overridden by a configuration
+   * parameter for unit testing.
+   */
+
+  private final long memoryLimit;
+
+  /**
+   * Estimated size of the records for this query, updated on each
+   * new batch received from upstream.
+   */
+
+  private int estimatedRowWidth;
+
+  /**
+   * Size of the merge batches that this operator produces. Generally
+   * the same as the merge batch size, unless low memory forces a smaller
+   * value.
+   */
+
+  private int expectedMergeBatchSize;
+
+  /**
+   * Estimate of the input batch size based on the largest batch seen
+   * thus far.
+   */
+  private int estimatedInputBatchSize;
+
+  /**
+   * Maximum memory level before spilling occurs. That is, we can buffer input
+   * batches in memory until we reach the level given by the buffer memory pool.
+   */
+
+  private long bufferMemoryLimit;
+
+  /**
+   * Maximum memory that can hold batches during the merge
+   * phase.
+   */
+
+  private long mergeMemoryLimit;
+
+  /**
+   * The target size for merge batches sent downstream.
+   */
+
+  private int preferredMergeBatchSize;
+
+  /**
+   * The configured size for each spill batch.
+   */
+  private int preferredSpillBatchSize;
+
+  /**
+   * Estimated number of rows that fit into a single spill batch.
+   */
+
+  private int spillBatchRowCount;
+
+  /**
+   * The estimated actual spill batch size which depends on the
+   * details of the data rows for any particular query.
+   */
+
+  private int expectedSpillBatchSize;
+
+  /**
+   * The number of records to add to each output batch sent to the
+   * downstream operator or spilled to disk.
+   */
+
+  private int mergeBatchRowCount;
+
+  private SortConfig config;
+
+  private int estimatedInputSize;
+
+  private boolean potentialOverflow;
+
+  public SortMemoryManager(SortConfig config, long memoryLimit) {
+    this.config = config;
+
+    // The maximum memory this operator can use as set by the
+    // operator definition (propagated to the allocator.)
+
+    if (config.maxMemory() > 0) {
+      this.memoryLimit = Math.min(memoryLimit, config.maxMemory());
+    } else {
+      this.memoryLimit = memoryLimit;
+    }
+
+    preferredSpillBatchSize = config.spillBatchSize();;
+    preferredMergeBatchSize = config.mergeBatchSize();
+  }
+
+  /**
+   * Update the data-driven memory use numbers including:
+   * <ul>
+   * <li>The average size of incoming records.</li>
+   * <li>The estimated spill and output batch size.</li>
+   * <li>The estimated number of average-size records per
+   * spill and output batch.</li>
+   * <li>The amount of memory set aside to hold the incoming
+   * batches before spilling starts.</li>
+   * </ul>
+   * <p>
+   * Under normal circumstances, the amount of memory available is much
+   * larger than the input, spill or merge batch sizes. The primary question
+   * is to determine how many input batches we can buffer during the load
+   * phase, and how many spill batches we can merge during the merge
+   * phase.
+   *
+   * @param batchSize the overall size of the current batch received from
+   * upstream
+   * @param batchRowWidth the average width in bytes (including overhead) of
+   * rows in the current input batch
+   * @param batchRowCount the number of actual (not filtered) records in
+   * that upstream batch
+   */
+
+  public void updateEstimates(int batchSize, int batchRowWidth, int batchRowCount) {
+
+    // The record count should never be zero, but better safe than sorry...
+
+    if (batchRowCount == 0) {
+      return; }
+
+
+    // Update input batch estimates.
+    // Go no further if nothing changed.
+
+    if (! updateInputEstimates(batchSize, batchRowWidth, batchRowCount)) {
+      return;
+    }
+
+    updateSpillSettings();
+    updateMergeSettings();
+    adjustForLowMemory();
+    logSettings(batchRowCount);
+  }
+
+  private boolean updateInputEstimates(int batchSize, int batchRowWidth, int batchRowCount) {
+
+    // The row width may end up as zero if all fields are nulls or some
+    // other unusual situation. In this case, assume a width of 10 just
+    // to avoid lots of special case code.
+
+    if (batchRowWidth == 0) {
+      batchRowWidth = 10;
+    }
+
+    // We know the batch size and number of records. Use that to estimate
+    // the average record size. Since a typical batch has many records,
+    // the average size is a fairly good estimator. Note that the batch
+    // size includes not just the actual vector data, but any unused space
+    // resulting from power-of-two allocation. This means that we don't
+    // have to do size adjustments for input batches as we will do below
+    // when estimating the size of other objects.
+
+    // Record sizes may vary across batches. To be conservative, use
+    // the largest size observed from incoming batches.
+
+    int origRowEstimate = estimatedRowWidth;
+    estimatedRowWidth = Math.max(estimatedRowWidth, batchRowWidth);
+
+    // Maintain an estimate of the incoming batch size: the largest
+    // batch yet seen. Used to reserve memory for the next incoming
+    // batch. Because we are using the actual observed batch size,
+    // the size already includes overhead due to power-of-two rounding.
+
+    long origInputBatchSize = estimatedInputBatchSize;
+    estimatedInputBatchSize = Math.max(estimatedInputBatchSize, batchSize);
+
+    // Estimate the total size of each incoming batch plus sv2. Note that, due
+    // to power-of-two rounding, the allocated sv2 size might be twice the data size.
+
+    estimatedInputSize = estimatedInputBatchSize + 4 * batchRowCount;
+
+    // Return whether anything changed.
+
+    return estimatedRowWidth != origRowEstimate || estimatedInputBatchSize != origInputBatchSize;
+  }
+
+  /**
+   * Determine the number of records to spill per spill batch. The goal is to
+   * spill batches of either 64K records, or as many records as fit into the
+   * amount of memory dedicated to each spill batch, whichever is less.
+   */
+
+  private void updateSpillSettings() {
+
+    spillBatchRowCount = rowsPerBatch(preferredSpillBatchSize);
+
+    // Compute the actual spill batch size which may be larger or smaller
+    // than the preferred size depending on the row width. Double the estimated
+    // memory needs to allow for power-of-two rounding.
+
+    expectedSpillBatchSize = batchForRows(spillBatchRowCount);
+
+    // Determine the minimum memory needed for spilling. Spilling is done just
+    // before accepting a spill batch, so we must spill if we don't have room for a
+    // (worst case) input batch. To spill, we need room for the spill batch created
+    // by merging the batches already in memory.
+
+    bufferMemoryLimit = memoryLimit - expectedSpillBatchSize;
+  }
+
+  /**
+   * Determine the number of records per batch per merge step. The goal is to
+   * merge batches of either 64K records, or as many records as fit into the
+   * amount of memory dedicated to each merge batch, whichever is less.
+   */
+
+  private void updateMergeSettings() {
+
+    mergeBatchRowCount = rowsPerBatch(preferredMergeBatchSize);
+    expectedMergeBatchSize = batchForRows(mergeBatchRowCount);
+
+    // The merge memory pool assumes we can spill all input batches. The memory
+    // available to hold spill batches for merging is total memory minus the
+    // expected output batch size.
+
+    mergeMemoryLimit = memoryLimit - expectedMergeBatchSize;
+  }
+
+  /**
+   * In a low-memory situation we have to approach the memory assignment
+   * problem from a different angle. Memory is low enough that we can't
+   * fit the incoming batches (of a size decided by the upstream operator)
+   * and our usual spill or merge batch sizes. Instead, we have to
+   * determine the largest spill and merge batch sizes possible given
+   * the available memory, input batch size and row width. We shrink the
+   * sizes of the batches we control to try to make things fit into limited
+   * memory. At some point, however, if we cannot fit even two input
+   * batches and even the smallest merge match, then we will run into an
+   * out-of-memory condition and we log a warning.
+   * <p>
+   * Note that these calculations are a bit crazy: it is Drill that
+   * decided to allocate the small memory, it is Drill that created the
+   * large incoming batches, and so it is Drill that created the low
+   * memory situation. Over time, a better fix for this condition is to
+   * control memory usage at the query level so that the sort is guaranteed
+   * to have sufficient memory. But, since we don't yet have the luxury
+   * of making such changes, we just live with the situation as we find
+   * it.
+   */
+
+  private void adjustForLowMemory() {
+
+    long loadHeadroom = bufferMemoryLimit - 2 * estimatedInputSize;
+    long mergeHeadroom = mergeMemoryLimit - 2 * expectedSpillBatchSize;
+    if (loadHeadroom >= 0  &&  mergeHeadroom >= 0) {
+      return;
+    }
+
+    lowMemorySpillBatchSize();
+    lowMemoryMergeBatchSize();
+
+    // Sanity check: if we've been given too little memory to make progress,
+    // issue a warning but proceed anyway. Should only occur if something is
+    // configured terribly wrong.
+
+    long minNeeds = 2 * estimatedInputSize + expectedSpillBatchSize;
+    if (minNeeds > memoryLimit) {
+      ExternalSortBatch.logger.warn("Potential memory overflow during load phase! " +
+          "Minimum needed = {} bytes, actual available = {} bytes",
+          minNeeds, memoryLimit);
+      bufferMemoryLimit = 0;
+      potentialOverflow = true;
+    }
+
+    // Sanity check
+
+    minNeeds = 2 * expectedSpillBatchSize + expectedMergeBatchSize;
+    if (minNeeds > memoryLimit) {
+      ExternalSortBatch.logger.warn("Potential memory overflow during merge phase! " +
+          "Minimum needed = {} bytes, actual available = {} bytes",
+          minNeeds, memoryLimit);
+      mergeMemoryLimit = 0;
+      potentialOverflow = true;
+    }
+  }
+
+  /**
+   * If we are in a low-memory condition, then we might not have room for the
+   * default spill batch size. In that case, pick a smaller size based on
+   * the observation that we need two input batches and
+   * one spill batch to make progress.
+   */
+
+  private void lowMemorySpillBatchSize() {
+
+    // The "expected" size is with power-of-two rounding in some vectors.
+    // We later work backwards to the row count assuming average internal
+    // fragmentation.
+
+    // Must hold two input batches. Use (most of) the rest for the spill batch.
+
+    expectedSpillBatchSize = (int) (memoryLimit - 2 * estimatedInputSize);
+
+    // But, in the merge phase, we need two spill batches and one output batch.
+    // (Assume that the spill and merge are equal sizes.)
+    // Use 3/4 of memory for each batch (to allow power-of-two rounding:
+
+    expectedSpillBatchSize = (int) Math.min(expectedSpillBatchSize, memoryLimit/3);
+
+    // Never going to happen, but let's ensure we don't somehow create large batches.
+
+    expectedSpillBatchSize = Math.max(expectedSpillBatchSize, SortConfig.MIN_SPILL_BATCH_SIZE);
+
+    // Must hold at least one row to spill. That is, we can make progress if we
+    // create spill files that consist of single-record batches.
+
+    expectedSpillBatchSize = Math.max(expectedSpillBatchSize, estimatedRowWidth);
+
+    // Work out the spill batch count needed by the spill code. Allow room for
+    // power-of-two rounding.
+
+    spillBatchRowCount = rowsPerBatch(expectedSpillBatchSize);
+
+    // Finally, figure out when we must spill.
+
+    bufferMemoryLimit = memoryLimit - expectedSpillBatchSize;
+  }
+
+  /**
+   * For merge batch, we must hold at least two spill batches and
+   * one output batch.
+   */
+
+  private void lowMemoryMergeBatchSize() {
+    expectedMergeBatchSize = (int) (memoryLimit - 2 * expectedSpillBatchSize);
+    expectedMergeBatchSize = Math.max(expectedMergeBatchSize, SortConfig.MIN_MERGE_BATCH_SIZE);
+    expectedMergeBatchSize = Math.max(expectedMergeBatchSize, estimatedRowWidth);
+    mergeBatchRowCount = rowsPerBatch(expectedMergeBatchSize);
+    mergeMemoryLimit = memoryLimit - expectedMergeBatchSize;
+  }
+
+  /**
+   * Log the calculated values. Turn this on if things seem amiss.
+   * Message will appear only when the values change.
+   */
+
+  private void logSettings(int actualRecordCount) {
+
+    ExternalSortBatch.logger.debug("Input Batch Estimates: record size = {} bytes; input batch = {} bytes, {} records",
+                 estimatedRowWidth, estimatedInputBatchSize, actualRecordCount);
+    ExternalSortBatch.logger.debug("Merge batch size = {} bytes, {} records; spill file size: {} bytes",
+                 expectedSpillBatchSize, spillBatchRowCount, config.spillFileSize());
+    ExternalSortBatch.logger.debug("Output batch size = {} bytes, {} records",
+                 expectedMergeBatchSize, mergeBatchRowCount);
+    ExternalSortBatch.logger.debug("Available memory: {}, buffer memory = {}, merge memory = {}",
+                 memoryLimit, bufferMemoryLimit, mergeMemoryLimit);
+  }
+
+  public enum MergeAction { SPILL, MERGE, NONE }
+
+  public static class MergeTask {
+    public MergeAction action;
+    public int count;
+
+    public MergeTask(MergeAction action, int count) {
+      this.action = action;
+      this.count = count;
+    }
+  }
+
+  public MergeTask consolidateBatches(long allocMemory, int inMemCount, int spilledRunsCount) {
+
+    // Determine additional memory needed to hold one batch from each
+    // spilled run.
+
+    // If the on-disk batches and in-memory batches need more memory than
+    // is available, spill some in-memory batches.
+
+    if (inMemCount > 0) {
+      long mergeSize = spilledRunsCount * expectedSpillBatchSize;
+      if (allocMemory + mergeSize > mergeMemoryLimit) {
+        return new MergeTask(MergeAction.SPILL, 0);
+      }
+    }
+
+    // Maximum batches that fit into available memory.
+
+    int mergeLimit = (int) ((mergeMemoryLimit - allocMemory) / expectedSpillBatchSize);
+
+    // Can't merge more than the merge limit.
+
+    mergeLimit = Math.min(mergeLimit, config.mergeLimit());
+
+    // How many batches to merge?
+
+    int mergeCount = spilledRunsCount - mergeLimit;
+    if (mergeCount <= 0) {
+      return new MergeTask(MergeAction.NONE, 0);
+    }
+
+    // We will merge. This will create yet another spilled
+    // run. Account for that.
+
+    mergeCount += 1;
+
+    // Must merge at least 2 batches to make progress.
+    // This is the the (at least one) excess plus the allowance
+    // above for the new one.
+
+    // Can't merge more than the limit.
+
+    mergeCount = Math.min(mergeCount, config.mergeLimit());
+
+    // Do the merge, then loop to try again in case not
+    // all the target batches spilled in one go.
+
+    return new MergeTask(MergeAction.MERGE, mergeCount);
+  }
+
+  /**
+   * Compute the number of rows per batch assuming that the batch is
+   * subject to average internal fragmentation due to power-of-two
+   * rounding on vectors.
+   * <p>
+   * <pre>[____|__$__]</pre>
+   * In the above, the brackets represent the whole vector. The
+   * first half is always full. When the first half filled, the second
+   * half was allocated. On average, the second half will be half full.
+   *
+   * @param batchSize expected batch size, including internal fragmentation
+   * @return number of rows that fit into the batch
+   */
+
+  private int rowsPerBatch(int batchSize) {
+    int rowCount = batchSize * 3 / 4 / estimatedRowWidth;
+    return Math.max(1, Math.min(rowCount, Character.MAX_VALUE));
+  }
+
+  /**
+   * Compute the expected number of rows that fit into a given size
+   * batch, accounting for internal fragmentation due to power-of-two
+   * rounding on vector allocations.
+   *
+   * @param rowCount the desired number of rows in the batch
+   * @return the size of resulting batch, including power-of-two
+   * rounding.
+   */
+
+  private int batchForRows(int rowCount) {
+    return estimatedRowWidth * rowCount * 4 / 3;
+  }
+
+  // Must spill if we are below the spill point (the amount of memory
+  // needed to do the minimal spill.)
+
+  public boolean isSpillNeeded(long allocatedBytes, int incomingSize) {
+    return allocatedBytes + incomingSize >= bufferMemoryLimit;
+  }
+
+  public boolean hasMemoryMergeCapacity(long allocatedBytes, long neededForInMemorySort) {
+    return (freeMemory(allocatedBytes) >= neededForInMemorySort);
+  }
+
+  public long freeMemory(long allocatedBytes) {
+    return memoryLimit - allocatedBytes;
+  }
+
+  public long getMergeMemoryLimit() { return mergeMemoryLimit; }
+  public int getSpillBatchRowCount() { return spillBatchRowCount; }
+  public int getMergeBatchRowCount() { return mergeBatchRowCount; }
+
+  // Primarily for testing
+
+  @VisibleForTesting
+  public long getMemoryLimit() { return memoryLimit; }
+  @VisibleForTesting
+  public int getRowWidth() { return estimatedRowWidth; }
+  @VisibleForTesting
+  public int getInputBatchSize() { return estimatedInputBatchSize; }
+  @VisibleForTesting
+  public int getPreferredSpillBatchSize() { return preferredSpillBatchSize; }
+  @VisibleForTesting
+  public int getPreferredMergeBatchSize() { return preferredMergeBatchSize; }
+  @VisibleForTesting
+  public int getSpillBatchSize() { return expectedSpillBatchSize; }
+  @VisibleForTesting
+  public int getMergeBatchSize() { return expectedMergeBatchSize; }
+  @VisibleForTesting
+  public long getBufferMemoryLimit() { return bufferMemoryLimit; }
+  @VisibleForTesting
+  public boolean mayOverflow() { return potentialOverflow; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
new file mode 100644
index 0000000..d51e007
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
@@ -0,0 +1,97 @@
+/*
+ * 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 org.apache.drill.exec.ops.OperatorStatReceiver;
+
+public class SortMetrics {
+
+  private int peakBatchCount = -1;
+  private int inputRecordCount = 0;
+  private int inputBatchCount = 0; // total number of batches received so far
+
+  /**
+   * Sum of the total number of bytes read from upstream.
+   * This is the raw memory bytes, not actual data bytes.
+   */
+
+  private long totalInputBytes;
+
+  /**
+   * Tracks the minimum amount of remaining memory for use
+   * in populating an operator metric.
+   */
+
+  private long minimumBufferSpace;
+  private OperatorStatReceiver stats;
+  private int spillCount;
+  private int mergeCount;
+  private long writeBytes;
+
+  public SortMetrics(OperatorStatReceiver stats) {
+    this.stats = stats;
+  }
+
+  public void updateInputMetrics(int rowCount, int batchSize) {
+    inputRecordCount += rowCount;
+    inputBatchCount++;
+    totalInputBytes += batchSize;
+  }
+
+  public void updateMemory(long freeMem) {
+
+    if (minimumBufferSpace == 0) {
+      minimumBufferSpace = freeMem;
+    } else {
+      minimumBufferSpace = Math.min(minimumBufferSpace, freeMem);
+    }
+    stats.setLongStat(ExternalSortBatch.Metric.MIN_BUFFER, minimumBufferSpace);
+  }
+
+  public int getInputRowCount() { return inputRecordCount; }
+  public long getInputBatchCount() { return inputBatchCount; }
+  public long getInputBytes() { return totalInputBytes; }
+
+  public void updatePeakBatches(int bufferedBatchCount) {
+    if (peakBatchCount < bufferedBatchCount) {
+      peakBatchCount = bufferedBatchCount;
+      stats.setLongStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY, peakBatchCount);
+    }
+  }
+
+  public void incrMergeCount() {
+    stats.addLongStat(ExternalSortBatch.Metric.MERGE_COUNT, 1);
+    mergeCount++;
+  }
+
+  public void incrSpillCount() {
+    stats.addLongStat(ExternalSortBatch.Metric.SPILL_COUNT, 1);
+    spillCount++;
+  }
+
+  public void updateWriteBytes(long writeBytes) {
+    stats.setDoubleStat(ExternalSortBatch.Metric.SPILL_MB,
+        writeBytes / 1024.0D / 1024.0);
+    this.writeBytes = writeBytes;
+  }
+
+  public int getSpillCount() { return spillCount; }
+  public int getMergeCount() { return mergeCount; }
+  public long getWriteBytes() { return writeBytes; }
+  public int getPeakBatchCount() { return peakBatchCount; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
new file mode 100644
index 0000000..4231cf4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
@@ -0,0 +1,92 @@
+/*
+ * 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 org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.config.ExternalSort;
+import org.apache.drill.exec.physical.impl.xsort.SingleBatchSorter;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+
+/**
+ * Single-batch sorter using a generated implementation based on the
+ * schema and sort specification. The generated sorter is reused
+ * across batches. The sorter must be closed at each schema change
+ * so that the sorter will generate a new implementation against
+ * the changed schema.
+ */
+
+public class SorterWrapper extends BaseSortWrapper {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SorterWrapper.class);
+
+  /**
+   * 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 SorterWrapper(OperExecContext opContext) {
+    super(opContext);
+  }
+
+  public void sortBatch(VectorContainer convertedBatch, SelectionVector2 sv2) {
+
+    SingleBatchSorter sorter = getSorter(convertedBatch);
+    try {
+      sorter.setup(context, sv2, convertedBatch);
+      sorter.sort(sv2);
+    } catch (SchemaChangeException e) {
+      convertedBatch.clear();
+      throw UserException.unsupportedError(e)
+            .message("Unexpected schema change.")
+            .build(logger);
+    }
+  }
+
+  public void close() {
+    sorter = null;
+  }
+
+  private SingleBatchSorter getSorter(VectorAccessible batch) {
+    if (sorter == null) {
+      sorter = newSorter(batch);
+    }
+    return sorter;
+  }
+
+  private SingleBatchSorter newSorter(VectorAccessible batch) {
+    CodeGenerator<SingleBatchSorter> cg = CodeGenerator.get(
+        SingleBatchSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(),
+        context.getOptionSet());
+    ClassGenerator<SingleBatchSorter> g = cg.getRoot();
+    cg.plainJavaCapable(true);
+    // Uncomment out this line to debug the generated code.
+  cg.saveCodeForDebugging(true);
+
+    generateComparisons(g, batch, logger);
+    return getInstance(cg, logger);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
new file mode 100644
index 0000000..a6042c6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
@@ -0,0 +1,235 @@
+/*
+ * 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.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.impl.spill.SpillSet;
+import org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.SpilledRun;
+import org.apache.drill.exec.physical.impl.xsort.managed.SortImpl.SortResults;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.VectorContainer;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Represents the set of spilled batches, including methods to spill and/or
+ * merge a set of batches to produce a new spill file.
+ */
+
+public class SpilledRuns {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SpilledRuns.class);
+
+  /**
+   * Manages the set of spill directories and files.
+   */
+
+  private final SpillSet spillSet;
+  private final LinkedList<BatchGroup.SpilledRun> spilledRuns = Lists.newLinkedList();
+
+  /**
+   * Manages the copier used to merge a collection of batches into
+   * a new set of batches.
+   */
+
+  private final PriorityQueueCopierWrapper copierHolder;
+  private BatchSchema schema;
+
+  private final OperExecContext context;
+
+  public SpilledRuns(OperExecContext opContext, SpillSet spillSet, PriorityQueueCopierWrapper copier) {
+    this.context = opContext;
+    this.spillSet = spillSet;
+//    copierHolder = new PriorityQueueCopierWrapper(opContext);
+    copierHolder = copier;
+  }
+
+  public void setSchema(BatchSchema schema) {
+    this.schema = schema;
+    for (BatchGroup b : spilledRuns) {
+      b.setSchema(schema);
+    }
+    copierHolder.close();
+  }
+
+  public int size() { return spilledRuns.size(); }
+  public boolean hasSpilled() { return spillSet.hasSpilled(); }
+  public long getWriteBytes() { return spillSet.getWriteBytes(); }
+
+  public static List<BatchGroup> prepareSpillBatches(LinkedList<? extends BatchGroup> source, int spillCount) {
+    List<BatchGroup> batchesToSpill = Lists.newArrayList();
+    spillCount = Math.min(source.size(), spillCount);
+    assert spillCount > 0 : "Spill count to mergeAndSpill must not be zero";
+    for (int i = 0; i < spillCount; i++) {
+      batchesToSpill.add(source.pollFirst());
+    }
+    return batchesToSpill;
+  }
+
+  public void mergeAndSpill(List<BatchGroup> batchesToSpill, int spillBatchRowCount) {
+    spilledRuns.add(safeMergeAndSpill(batchesToSpill, spillBatchRowCount));
+    logger.trace("Completed spill: memory = {}",
+        context.getAllocator().getAllocatedMemory());
+  }
+
+  public void mergeRuns(int targetCount, long mergeMemoryPool, int spillBatchRowCount) {
+
+    long allocated = context.getAllocator().getAllocatedMemory();
+    mergeMemoryPool -= context.getAllocator().getAllocatedMemory();
+    logger.trace("Merging {} on-disk runs, alloc. memory = {}, avail. memory = {}",
+        targetCount, allocated, mergeMemoryPool);
+
+    // Determine the number of runs to merge. The count should be the
+    // target count. However, to prevent possible memory overrun, we
+    // double-check with actual spill batch size and only spill as much
+    // as fits in the merge memory pool.
+
+    int mergeCount = 0;
+    long mergeSize = 0;
+    for (SpilledRun run : spilledRuns) {
+      long batchSize = run.getBatchSize();
+      if (mergeSize + batchSize > mergeMemoryPool) {
+        break;
+      }
+      mergeSize += batchSize;
+      mergeCount++;
+      if (mergeCount == targetCount) {
+        break;
+      }
+    }
+
+    // Must always spill at least 2, even if this creates an over-size
+    // spill file. But, if this is a final consolidation, we may have only
+    // a single batch.
+
+    mergeCount = Math.max(mergeCount, 2);
+    mergeCount = Math.min(mergeCount, spilledRuns.size());
+
+    // Do the actual spill.
+
+    List<BatchGroup> batchesToSpill = prepareSpillBatches(spilledRuns, mergeCount);
+    mergeAndSpill(batchesToSpill, spillBatchRowCount);
+  }
+
+  private BatchGroup.SpilledRun safeMergeAndSpill(List<? extends BatchGroup> batchesToSpill, int spillBatchRowCount) {
+    try {
+      return doMergeAndSpill(batchesToSpill, spillBatchRowCount);
+    }
+    // If error is a User Exception, just use as is.
+
+    catch (UserException ue) { throw ue; }
+    catch (Throwable ex) {
+      throw UserException.resourceError(ex)
+            .message("External Sort encountered an error while spilling to disk")
+            .build(logger);
+    }
+  }
+
+  private BatchGroup.SpilledRun doMergeAndSpill(List<? extends BatchGroup> batchesToSpill, int spillBatchRowCount) throws Throwable {
+
+    // Merge the selected set of matches and write them to the
+    // spill file. After each write, we release the memory associated
+    // with the just-written batch.
+
+    String outputFile = spillSet.getNextSpillFile();
+    BatchGroup.SpilledRun newGroup = null;
+    VectorContainer dest = new VectorContainer();
+    try (AutoCloseable ignored = AutoCloseables.all(batchesToSpill);
+         PriorityQueueCopierWrapper.BatchMerger merger = copierHolder.startMerge(schema, batchesToSpill, dest, spillBatchRowCount)) {
+      newGroup = new BatchGroup.SpilledRun(spillSet, outputFile, context.getAllocator());
+      logger.trace("Spilling {} batches, into spill batches of {} rows, to {}",
+          batchesToSpill.size(), spillBatchRowCount, outputFile);
+
+      // The copier will merge records from the buffered batches into
+      // the outputContainer up to targetRecordCount number of rows.
+      // The actual count may be less if fewer records are available.
+
+      while (merger.next()) {
+
+        // Add a new batch of records (given by merger.getOutput()) to the spill
+        // file.
+        //
+        // note that addBatch also clears the merger's output container
+
+        newGroup.addBatch(dest);
+      }
+      context.injectChecked(ExternalSortBatch.INTERRUPTION_WHILE_SPILLING, IOException.class);
+      newGroup.closeOutputStream();
+      logger.trace("Spilled {} output batches, each of {} by bytes, {} records to {}",
+                   merger.getBatchCount(), merger.getRecordCount(),
+                   merger.getEstBatchSize(), outputFile);
+      newGroup.setBatchSize(merger.getEstBatchSize());
+      return newGroup;
+    } catch (Throwable e) {
+      // we only need to clean up newGroup if spill failed
+      try {
+        if (newGroup != null) {
+          AutoCloseables.close(e, newGroup);
+        }
+      } catch (Throwable t) { /* close() may hit the same IO issue; just ignore */ }
+
+      throw e;
+    }
+  }
+
+  public SortResults finalMerge(List<? extends BatchGroup> bufferedBatches, VectorContainer container, int mergeRowCount) {
+    List<BatchGroup> allBatches = new LinkedList<>();
+    allBatches.addAll(bufferedBatches);
+    bufferedBatches.clear();
+    allBatches.addAll(spilledRuns);
+    spilledRuns.clear();
+    logger.debug("Starting merge phase. Runs = {}, Alloc. memory = {}",
+        allBatches.size(), context.getAllocator().getAllocatedMemory());
+    return copierHolder.startMerge(schema, allBatches, container, mergeRowCount);
+  }
+
+  public void close() {
+    if (spillSet.getWriteBytes() > 0) {
+      logger.debug("End of sort. Total write bytes: {}, Total read bytes: {}",
+                   spillSet.getWriteBytes(), spillSet.getWriteBytes());
+    }
+    RuntimeException ex = null;
+    try {
+      if (spilledRuns != null) {
+        BatchGroup.closeAll(spilledRuns);
+        spilledRuns.clear();
+      }
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    try {
+      copierHolder.close();
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    try {
+      spillSet.close();
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    if (ex != null) {
+      throw ex;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 8aedaf6..146df1f 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -34,11 +34,11 @@ drill {
     annotations += org.apache.drill.exec.expr.annotations.FunctionTemplate
 
     packages : ${?drill.classpath.scanning.packages} [
-      org.apache.drill.exec.expr,
-      org.apache.drill.exec.physical,
-      org.apache.drill.exec.store,
-      org.apache.drill.exec.rpc.user.security,
-      org.apache.drill.exec.rpc.security
+          org.apache.drill.exec.expr,
+          org.apache.drill.exec.physical,
+          org.apache.drill.exec.store,
+          org.apache.drill.exec.rpc.user.security,
+          org.apache.drill.exec.rpc.security
     ]
   }
 }
@@ -241,19 +241,19 @@ drill.exec: {
       // 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
+      disable_managed: true,
       // Limit on the number of batches buffered in memory.
       // Primarily for testing.
       // 0 = unlimited
-      batch_limit: 0
+      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
+      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
+      merge_limit: 0,
       spill: {
         // Deprecated for managed xsort; used only by legacy xsort
         group.size: 40000,


[5/5] drill git commit: DRILL-5325: Unit tests for the managed sort

Posted by pr...@apache.org.
DRILL-5325: Unit tests for the managed sort

Uses the sub-operator test framework (DRILL-5318), including the test
row set abstraction (DRILL-5323) to enable unit testing of the
“managed” external sort. This PR allows early review of the code, but
cannot be pulled until the dependencies (mentioned above) are pulled.

Refactors the external sort code into small chunks that can be unit
tested, then “wraps” that code in tests for all interesting data types,
record batch sizes, and so on.

Refactors some of the operator definitions to more easily allow
programmatic setup in the unit tests.

Fixes a number of bugs discovered by the unit tests. The biggest
changes were in the new code: the code that computes spilling and
merging based on memory levels.

Otherwise, although GitHub will show many files change, most of the
changes are simply moving blocks of code around to create smaller units
that can be tested independently.

Includes a refactoring of the code that does spilling, along with a
complete set of low-level unit tests.

Excludes long-running sort tests.

Defines a test category for long-running tests.

First attempt to provide a way to run such tests from Maven.

closes #808


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/90f43bff
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/90f43bff
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/90f43bff

Branch: refs/heads/master
Commit: 90f43bff7a01eaaee6c8861137759b05367dfcf3
Parents: c16e5f8
Author: Paul Rogers <pr...@maprtech.com>
Authored: Thu Apr 6 13:57:19 2017 -0700
Committer: Paul Rogers <pr...@maprtech.com>
Committed: Wed Jun 21 10:02:16 2017 -0700

----------------------------------------------------------------------
 .../org/apache/drill/test/SecondaryTest.java    |   70 +
 exec/java-exec/pom.xml                          |    5 +
 .../org/apache/drill/exec/ExecConstants.java    |    2 -
 .../drill/exec/cache/VectorSerializer.java      |  121 ++
 .../drill/exec/physical/base/AbstractBase.java  |    2 +-
 .../drill/exec/physical/base/FragmentRoot.java  |    2 +-
 .../apache/drill/exec/physical/base/Root.java   |    2 +-
 .../exec/physical/config/ExternalSort.java      |    4 +-
 .../impl/aggregate/HashAggTemplate.java         |    2 +-
 .../impl/sort/SortRecordBatchBuilder.java       |   12 +-
 .../exec/physical/impl/spill/SpillSet.java      |   35 +-
 .../impl/svremover/CopierTemplate2.java         |    6 +-
 .../exec/physical/impl/xsort/BatchGroup.java    |    2 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |    6 +-
 .../impl/xsort/managed/BaseSortWrapper.java     |   90 ++
 .../impl/xsort/managed/BaseWrapper.java         |   53 +
 .../physical/impl/xsort/managed/BatchGroup.java |   76 +-
 .../impl/xsort/managed/BufferedBatches.java     |  232 ++++
 .../impl/xsort/managed/CopierHolder.java        |  322 -----
 .../impl/xsort/managed/ExternalSortBatch.java   | 1226 ++----------------
 .../impl/xsort/managed/MSortTemplate.java       |   26 +-
 .../physical/impl/xsort/managed/MSorter.java    |    6 +-
 .../physical/impl/xsort/managed/MergeSort.java  |  167 ---
 .../impl/xsort/managed/MergeSortWrapper.java    |  261 ++++
 .../xsort/managed/OperatorCodeGenerator.java    |  259 ----
 .../impl/xsort/managed/PriorityQueueCopier.java |    3 +-
 .../managed/PriorityQueueCopierTemplate.java    |   74 +-
 .../managed/PriorityQueueCopierWrapper.java     |  341 +++++
 .../physical/impl/xsort/managed/SortConfig.java |  121 ++
 .../physical/impl/xsort/managed/SortImpl.java   |  491 +++++++
 .../impl/xsort/managed/SortMemoryManager.java   |  513 ++++++++
 .../impl/xsort/managed/SortMetrics.java         |   97 ++
 .../impl/xsort/managed/SorterWrapper.java       |   92 ++
 .../impl/xsort/managed/SpilledRuns.java         |  235 ++++
 .../src/main/resources/drill-module.conf        |   18 +-
 .../exec/cache/TestBatchSerialization.java      |  216 +++
 .../drill/exec/cache/TestWriteToDisk.java       |    2 +-
 .../physical/impl/xsort/TestExternalSort.java   |    3 +
 .../impl/xsort/TestSimpleExternalSort.java      |   69 +-
 .../impl/xsort/TestSortSpillWithException.java  |   72 +-
 .../impl/xsort/managed/SortTestUtilities.java   |  132 ++
 .../physical/impl/xsort/managed/TestCopier.java |  377 ++++++
 .../xsort/managed/TestExternalSortExec.java     |  188 +++
 .../managed/TestExternalSortInternals.java      |  632 +++++++++
 .../impl/xsort/managed/TestSortImpl.java        |  609 +++++++++
 .../physical/impl/xsort/managed/TestSorter.java |  605 +++++++++
 .../apache/drill/exec/memory/BaseAllocator.java |   25 +-
 .../drill/common/expression/FieldReference.java |    9 +-
 .../drill/common/expression/SchemaPath.java     |    5 +-
 .../apache/drill/common/logical/data/Order.java |   73 +-
 pom.xml                                         |    3 +-
 51 files changed, 5871 insertions(+), 2123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/common/src/main/java/org/apache/drill/test/SecondaryTest.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/test/SecondaryTest.java b/common/src/main/java/org/apache/drill/test/SecondaryTest.java
new file mode 100644
index 0000000..6b9a187
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/test/SecondaryTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.test;
+
+/**
+ * Label for Drill secondary tests. A secondary test is one that is omitted from
+ * the normal Drill build because:
+ * <ul>
+ * <li>It is slow</li>
+ * <li>It tests particular functionality which need not be tested on every
+ * build.</li>
+ * <li>It is old, but still worth running once in a while.</li>
+ * <li>It requires specialized setup and/or runs on specific platforms.</li>
+ * </ul>
+ *
+ * To mark a test as secondary, do either:<pre><code>
+ * {@literal @}Category(SecondaryTest.class)
+ * class MyTest {
+ *    ...
+ * }
+ * </pre></code>Or:<pre><code>
+ * class MyTest {
+ *   {@literal @}Category(SecondaryTest.class)
+ *   public void slowTest() { ... }
+ * }
+ * </code></pre>
+ * Maven is configured as follows:<pre><code>
+ *    &lt;plugin>
+ *      &lt;artifactId>maven-surefire-plugin&lt;/artifactId>
+ *      ...
+ *      &lt;configuration>
+ *        ...
+ *        &lt;excludedGroups>org.apache.drill.test.SecondaryTest&lt;/excludedGroups>
+ *      &lt;/configuration>
+ *      ...
+ *    &lt;/plugin></code></pre>
+ *  To run all tests (including the secondary tests) (preliminary):<pre><code>
+ *  > mvn surefire:test -Dgroups=org.apache.drill.test.SecondaryTest -DexcludedGroups=</code></pre>
+ *  The above says to run the secondary test and exclude nothing. The exclusion
+ *  is required to override the default exclusions: skip that parameter and Maven will
+ *  blindly try to run all tests annotated with the SecondaryTest category except
+ *  for those annotated with the SecondaryTest category, which is not very helpful...
+ *  <p>
+ *  Note that <tt>java-exec</tt> (only) provides a named execution to run large tests:
+ *  <p>
+ *  <tt>mvn surefire:test@include-large-tests</tt>
+ *  <p>
+ *  However, the above does not work. Nor did it work to include the category in
+ *  a profile as described earlier. At present, there is no known way to run just
+ *  the secondary tests from Maven. Sorry...
+ */
+
+public interface SecondaryTest {
+  // Junit category marker
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 3b1ac4c..cd287aa 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -532,6 +532,7 @@
                   <includes>
                     <include>**/TestLargeFileCompilation.java</include>
                   </includes>
+                  <groups>org.apache.drill.test.SecondaryTest</groups>
                 </configuration>
               </execution>
             </executions>
@@ -732,9 +733,13 @@
             <phase>test</phase>
             <goals><goal>test</goal></goals>
             <configuration>
+              <!--  Legacy: exclude by file name -->
               <excludes>
                 <exclude>**/TestLargeFileCompilation.java</exclude>
               </excludes>
+              <!-- Modern: include using the @Category annotation.
+                   See the Javadoc for SecondaryTest for details. -->
+              <excludedGroups>org.apache.drill.test.SecondaryTest</excludedGroups>
             </configuration>
           </execution>
         </executions>

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 537377d..e2782e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -84,8 +84,6 @@ public interface ExecConstants {
   String EXTERNAL_SORT_SPILL_BATCH_SIZE = "drill.exec.sort.external.spill.spill_batch_size";
   String EXTERNAL_SORT_MERGE_BATCH_SIZE = "drill.exec.sort.external.spill.merge_batch_size";
   String EXTERNAL_SORT_MAX_MEMORY = "drill.exec.sort.external.mem_limit";
-
-  // Used only by the "unmanaged" sort.
   String EXTERNAL_SORT_BATCH_LIMIT = "drill.exec.sort.external.batch_limit";
 
   // External Sort Runtime options

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
new file mode 100644
index 0000000..eeef9e5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.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.cache;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+
+/**
+ * Serializes vector containers to an output stream or from
+ * an input stream.
+ */
+
+public class VectorSerializer {
+
+  /**
+   * Writes multiple VectorAccessible or VectorContainer
+   * objects to an output stream.
+   */
+
+  public static class Writer {
+
+    private final OutputStream stream;
+    private final BufferAllocator allocator;
+    private boolean retain;
+    private long timeNs;
+
+    public Writer(BufferAllocator allocator, OutputStream stream) {
+      this.allocator = allocator;
+      this.stream = stream;
+    }
+
+    public Writer retain() {
+      retain = true;
+      return this;
+    }
+
+    public Writer write(VectorAccessible va) throws IOException {
+      return write(va, null);
+    }
+
+    @SuppressWarnings("resource")
+    public Writer write(VectorAccessible va, SelectionVector2 sv2) throws IOException {
+      WritableBatch batch = WritableBatch.getBatchNoHVWrap(
+          va.getRecordCount(), va, sv2 != null);
+      return write(batch, sv2);
+    }
+
+    public Writer write(WritableBatch batch, SelectionVector2 sv2) throws IOException {
+      VectorAccessibleSerializable vas;
+      if (sv2 == null) {
+        vas = new VectorAccessibleSerializable(batch, allocator);
+      } else {
+        vas = new VectorAccessibleSerializable(batch, sv2, allocator);
+      }
+      if (retain) {
+        vas.writeToStreamAndRetain(stream);
+      } else {
+        vas.writeToStream(stream);
+      }
+      timeNs += vas.getTimeNs();
+      return this;
+    }
+
+    public long timeNs() { return timeNs; }
+  }
+
+  /**
+   * Read one or more vector containers from an input stream.
+   */
+
+  public static class Reader {
+    private final InputStream stream;
+    private long timeNs;
+    private final VectorAccessibleSerializable vas;
+
+    public Reader(BufferAllocator allocator, InputStream stream) {
+      this.stream = stream;
+      vas = new VectorAccessibleSerializable(allocator);
+    }
+
+    public VectorContainer read() throws IOException {
+      vas.readFromStream(stream);
+      timeNs = vas.getTimeNs();
+      return vas.get();
+    }
+
+    public SelectionVector2 sv2() { return vas.getSv2(); }
+
+    public long timeNs() { return timeNs; }
+  }
+
+  public static Writer writer(BufferAllocator allocator, OutputStream stream) {
+    return new Writer(allocator, stream);
+  }
+
+  public static Reader reader(BufferAllocator allocator, InputStream stream) {
+    return new Reader(allocator, stream);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
index 6f42250..b4c8536 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
@@ -24,7 +24,7 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.google.common.base.Preconditions;
 
-public abstract class AbstractBase implements PhysicalOperator{
+public abstract class AbstractBase implements PhysicalOperator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
 
   public static long INIT_ALLOCATION = 1_000_000L;

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
index 1721fcf..32910c3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
@@ -21,5 +21,5 @@ package org.apache.drill.exec.physical.base;
 /**
  * Describes the root operation within a particular Fragment. This includes things Sender nodes.
  */
-public interface FragmentRoot extends FragmentLeaf{
+public interface FragmentRoot extends FragmentLeaf {
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
index bd280ae..e562240 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
@@ -20,5 +20,5 @@ package org.apache.drill.exec.physical.base;
 /**
  * Marker interface describe the root of a query plan.  Currently, this is constrained to Screen.
  */
-public interface Root extends FragmentRoot{
+public interface Root extends FragmentRoot {
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
index cb9679d..f0e88b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
@@ -31,10 +31,12 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 public class ExternalSort extends Sort {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExternalSort.class);
 
+  public static final long DEFAULT_SORT_ALLOCATION = 20_000_000;
+
   @JsonCreator
   public ExternalSort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<Ordering> orderings, @JsonProperty("reverse") boolean reverse) {
     super(child, orderings, reverse);
-    initialAllocation = 20_000_000;
+    initialAllocation = DEFAULT_SORT_ALLOCATION;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 38f0222..8393937 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -367,7 +367,7 @@ public abstract class HashAggTemplate implements HashAggregator {
       }
     }
 
-    spillSet = new SpillSet(context,hashAggrConfig, UserBitShared.CoreOperatorType.HASH_AGGREGATE);
+    spillSet = new SpillSet(context, hashAggrConfig);
     baseHashTable =
         new ChainedHashTable(htConfig, context, allocator, incoming, null /* no incoming probe */, outgoing);
     this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
index d46990f..cd52a91 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -98,6 +98,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     return true;
   }
 
+  @SuppressWarnings("resource")
   public void add(RecordBatchData rbd) {
     long batchBytes = getSize(rbd.getContainer());
     if (batchBytes == 0 && batches.size() > 0) {
@@ -140,7 +141,12 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     return batches.isEmpty();
   }
 
-  public void build(FragmentContext context, VectorContainer outputContainer) throws SchemaChangeException{
+  public void build(FragmentContext context, VectorContainer outputContainer) throws SchemaChangeException {
+    build(outputContainer);
+  }
+
+  @SuppressWarnings("resource")
+  public void build(VectorContainer outputContainer) throws SchemaChangeException {
     outputContainer.clear();
     if (batches.keySet().size() > 1) {
       throw new SchemaChangeException("Sort currently only supports a single schema.");
@@ -177,7 +183,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
       int index = 0;
       int recordBatchId = 0;
       for (RecordBatchData d : data) {
-        for (int i =0; i < d.getRecordCount(); i++, index++) {
+        for (int i = 0; i < d.getRecordCount(); i++, index++) {
           sv4.set(index, recordBatchId, (int) d.getSv2().getIndex(i));
         }
         // might as well drop the selection vector since we'll stop using it now.

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
index 87eebc6..5a02cee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
@@ -30,13 +30,13 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.HashAggregate;
+import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -283,7 +283,7 @@ public class SpillSet {
     private File baseDir;
 
     public LocalFileManager(String fsName) {
-      baseDir = new File(fsName.replace("file://", ""));
+      baseDir = new File(fsName.replace(FileSystem.DEFAULT_FS, ""));
     }
 
     @Override
@@ -357,30 +357,31 @@ public class SpillSet {
 
   private long writeBytes;
 
-  public SpillSet(FragmentContext context, PhysicalOperator popConfig, UserBitShared.CoreOperatorType optype) {
-    FragmentHandle handle = context.getHandle();
-    String operName = "Unknown";
+  public SpillSet(FragmentContext context, PhysicalOperator popConfig) {
+    this(context.getConfig(), context.getHandle(), popConfig);
+  }
+
+  public SpillSet(DrillConfig config, FragmentHandle handle, PhysicalOperator popConfig) {
+    String operName;
 
     // Set the spill options from the configuration
-    DrillConfig config = context.getConfig();
     String spillFs;
     List<String> dirList;
 
     // Set the operator name (used as part of the spill file name),
     // and set oper. specific options (the config file defaults to using the
     // common options; users may override those - per operator)
-    switch (optype) {
-      case EXTERNAL_SORT:
+    if (popConfig instanceof Sort) {
         operName = "Sort";
         spillFs = config.getString(ExecConstants.EXTERNAL_SORT_SPILL_FILESYSTEM);
         dirList = config.getStringList(ExecConstants.EXTERNAL_SORT_SPILL_DIRS);
-        break;
-      case HASH_AGGREGATE:
+    } else if (popConfig instanceof HashAggregate) {
         operName = "HashAgg";
         spillFs = config.getString(ExecConstants.HASHAGG_SPILL_FILESYSTEM);
         dirList = config.getStringList(ExecConstants.HASHAGG_SPILL_DIRS);
-        break;
-      default: // just use the common ones
+    } else {
+        // just use the common ones
+        operName = "Unknown";
         spillFs = config.getString(ExecConstants.SPILL_FILESYSTEM);
         dirList = config.getStringList(ExecConstants.SPILL_DIRS);
     }
@@ -406,7 +407,7 @@ public class SpillSet {
     // as a proxy for a non-production Drill setup.)
 
     boolean impersonationEnabled = config.getBoolean(ExecConstants.IMPERSONATION_ENABLED);
-    if (spillFs.startsWith("file:///") && ! impersonationEnabled) {
+    if (spillFs.startsWith(FileSystem.DEFAULT_FS) && ! impersonationEnabled) {
       fileManager = new LocalFileManager(spillFs);
     } else {
       fileManager = new HadoopFileManager(spillFs);
@@ -416,6 +417,10 @@ public class SpillSet {
         operName, handle.getMajorFragmentId(), popConfig.getOperatorId(), handle.getMinorFragmentId());
   }
 
+  public String getNextSpillFile() {
+    return getNextSpillFile(null);
+  }
+
   public String getNextSpillFile(String extraName) {
 
     // Identify the next directory from the round-robin list to
@@ -427,7 +432,7 @@ public class SpillSet {
     currSpillDirs.add(currSpillPath);
 
     String outputFile = Joiner.on("/").join(currSpillPath, "spill" + ++fileCount);
-    if ( extraName != null ) {
+    if (extraName != null) {
       outputFile += "_" + extraName;
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
index bdee8ae..96daf7f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/CopierTemplate2.java
@@ -65,9 +65,7 @@ public abstract class CopierTemplate2 implements Copier{
                                @Named("incoming") RecordBatch incoming,
                                @Named("outgoing") RecordBatch outgoing)
                        throws SchemaChangeException;
-  public abstract void doEval(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex)
+  public abstract void doEval(@Named("inIndex") int inIndex,
+                              @Named("outIndex") int outIndex)
                        throws SchemaChangeException;
-
-
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/BatchGroup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/BatchGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/BatchGroup.java
index 13f0dbe..09d6bae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/BatchGroup.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/BatchGroup.java
@@ -117,7 +117,7 @@ public class BatchGroup implements VectorAccessible, AutoCloseable {
     spilledBatches--;
     currentContainer.zeroVectors();
     Iterator<VectorWrapper<?>> wrapperIterator = c.iterator();
-    for (VectorWrapper w : currentContainer) {
+    for (VectorWrapper<?> w : currentContainer) {
       TransferPair pair = wrapperIterator.next().getValueVector().makeTransferPair(w.getValueVector());
       pair.transfer();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 25f05b3..39c662f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -346,9 +346,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
               if (unionTypeEnabled) {
                 this.schema = SchemaUtil.mergeSchemas(schema, incoming.getSchema());
               } else {
-                throw SchemaChangeException.schemaChanged("Schema changes not supported in External Sort. Please enable Union type",
-                    schema,
-                    incoming.getSchema());
+                throw new SchemaChangeException("Schema changes not supported in External Sort. Please enable Union type");
               }
             } else {
               schema = incoming.getSchema();
@@ -475,7 +473,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
           builder.add(rbd);
         }
 
-        builder.build(context, container);
+        builder.build(container);
         sv4 = builder.getSv4();
         mSorter = createNewMSorter();
         mSorter.setup(context, oAllocator, getSelectionVector4(), this.container);

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseSortWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseSortWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseSortWrapper.java
new file mode 100644
index 0000000..1f381b9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseSortWrapper.java
@@ -0,0 +1,90 @@
+/*
+ * 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 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.MappingSet;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.ClassGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.record.VectorAccessible;
+
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+
+/**
+ * Base wrapper for algorithms that use sort comparisons.
+ */
+
+public abstract class BaseSortWrapper extends BaseWrapper {
+
+  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);
+
+  public BaseSortWrapper(OperExecContext opContext) {
+    super(opContext);
+  }
+
+  protected void generateComparisons(ClassGenerator<?> g, VectorAccessible batch, org.slf4j.Logger logger)  {
+    g.setMappingSet(MAIN_MAPPING);
+
+    Sort popConfig = context.getOperatorDefn();
+    for (Ordering od : popConfig.getOrderings()) {
+      // first, we rewrite the evaluation stack for each side of the comparison.
+      ErrorCollector collector = new ErrorCollectorImpl();
+      final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector, context.getFunctionRegistry());
+      if (collector.hasErrors()) {
+        throw UserException.unsupportedError()
+              .message("Failure while materializing expression. " + collector.toErrorString())
+              .build(logger);
+      }
+      g.setMappingSet(LEFT_MAPPING);
+      HoldingContainer left = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
+      g.setMappingSet(RIGHT_MAPPING);
+      HoldingContainer right = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
+      g.setMappingSet(MAIN_MAPPING);
+
+      // next we wrap the two comparison sides and add the expression block for the comparison.
+      LogicalExpression fh =
+          FunctionGenerationHelper.getOrderingComparator(od.nullsSortHigh(), left, right,
+                                                         context.getFunctionRegistry());
+      HoldingContainer out = g.addExpr(fh, ClassGenerator.BlkCreateMode.FALSE);
+      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
+
+      if (od.getDirection() == Direction.ASCENDING) {
+        jc._then()._return(out.getValue());
+      }else{
+        jc._then()._return(out.getValue().minus());
+      }
+      g.rotateBlock();
+    }
+
+    g.rotateBlock();
+    g.getEvalBlock()._return(JExpr.lit(0));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseWrapper.java
new file mode 100644
index 0000000..e607f40
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BaseWrapper.java
@@ -0,0 +1,53 @@
+/*
+ * 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 org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.ops.OperExecContext;
+
+/**
+ * Base class for code-generation-based tasks.
+ */
+
+public abstract class BaseWrapper {
+
+  protected OperExecContext context;
+
+  public BaseWrapper(OperExecContext context) {
+    this.context = context;
+  }
+
+  protected <T> T getInstance(CodeGenerator<T> cg, org.slf4j.Logger logger) {
+    try {
+      return context.getImplementationClass(cg);
+    } catch (ClassTransformationException e) {
+      throw UserException.unsupportedError(e)
+            .message("Code generation error - likely code error.")
+            .build(logger);
+    } catch (IOException e) {
+      throw UserException.resourceError(e)
+            .message("IO Error during code generation.")
+            .build(logger);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
index 2e5d5b2..a74183c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
@@ -20,14 +20,14 @@ package org.apache.drill.exec.physical.impl.xsort.managed;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collection;
 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.cache.VectorSerializer;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.SchemaUtil;
@@ -36,7 +36,6 @@ 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;
 
@@ -78,15 +77,13 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
     private final SelectionVector2 sv2;
     private final int dataSize;
 
-    public InputBatch(VectorContainer container, SelectionVector2 sv2, OperatorContext context, int dataSize) {
-      super(container, context);
+    public InputBatch(VectorContainer container, SelectionVector2 sv2, BufferAllocator allocator, int dataSize) {
+      super(container, allocator);
       this.sv2 = sv2;
       this.dataSize = dataSize;
     }
 
-    public SelectionVector2 getSv2() {
-      return sv2;
-    }
+    public SelectionVector2 getSv2() { return sv2; }
 
     public int getDataSize() { return dataSize; }
 
@@ -110,14 +107,10 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
 
     @Override
     public void close() throws IOException {
-      try {
-        super.close();
-      }
-      finally {
-        if (sv2 != null) {
-          sv2.clear();
-        }
+      if (sv2 != null) {
+        sv2.clear();
       }
+      super.close();
     }
   }
 
@@ -152,24 +145,23 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
     private BufferAllocator allocator;
     private int spilledBatches;
     private long batchSize;
+    private VectorSerializer.Writer writer;
+    private VectorSerializer.Reader reader;
 
-    public SpilledRun(SpillSet spillSet, String path, OperatorContext context) throws IOException {
-      super(null, context);
+    public SpilledRun(SpillSet spillSet, String path, BufferAllocator allocator) throws IOException {
+      super(null, allocator);
       this.spillSet = spillSet;
       this.path = path;
-      this.allocator = context.getAllocator();
+      this.allocator = allocator;
       outputStream = spillSet.openForOutput(path);
+      writer = VectorSerializer.writer(allocator, outputStream);
     }
 
     public void addBatch(VectorContainer newContainer) throws IOException {
-      int recordCount = newContainer.getRecordCount();
-      @SuppressWarnings("resource")
-      WritableBatch batch = WritableBatch.getBatchNoHVWrap(recordCount, newContainer, false);
-      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
       Stopwatch watch = Stopwatch.createStarted();
-      outputBatch.writeToStream(outputStream);
+      writer.write(newContainer);
       newContainer.zeroVectors();
-      logger.trace("Wrote {} records in {} us", recordCount, watch.elapsed(TimeUnit.MICROSECONDS));
+      logger.trace("Wrote {} records in {} us", newContainer.getRecordCount(), watch.elapsed(TimeUnit.MICROSECONDS));
       spilledBatches++;
 
       // Hold onto the husk of the last added container so that we have a
@@ -184,6 +176,7 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
     }
 
     public long getBatchSize() { return batchSize; }
+    public String getPath() { return path; }
 
     @Override
     public int getNextIndex() {
@@ -216,19 +209,18 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
     private VectorContainer getBatch() throws IOException {
       if (inputStream == null) {
         inputStream = spillSet.openForInput(path);
+        reader = VectorSerializer.reader(allocator, inputStream);
       }
-      VectorAccessibleSerializable vas = new VectorAccessibleSerializable(allocator);
       Stopwatch watch = Stopwatch.createStarted();
-      vas.readFromStream(inputStream);
-      VectorContainer c =  vas.get();
+      VectorContainer c =  reader.read();
       if (schema != null) {
-        c = SchemaUtil.coerceContainer(c, schema, context);
+        c = SchemaUtil.coerceContainer(c, schema, allocator);
       }
       logger.trace("Read {} records in {} us", c.getRecordCount(), watch.elapsed(TimeUnit.MICROSECONDS));
       spilledBatches--;
       currentContainer.zeroVectors();
       Iterator<VectorWrapper<?>> wrapperIterator = c.iterator();
-      for (@SuppressWarnings("rawtypes") VectorWrapper w : currentContainer) {
+      for (VectorWrapper<?> w : currentContainer) {
         TransferPair pair = wrapperIterator.next().getValueVector().makeTransferPair(w.getValueVector());
         pair.transfer();
       }
@@ -279,6 +271,7 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
       spillSet.tallyReadBytes(readLength);
       inputStream.close();
       inputStream = null;
+      reader = null;
       logger.trace("Summary: Read {} bytes from {}", readLength, path);
     }
 
@@ -290,6 +283,7 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
       spillSet.tallyWriteBytes(writeSize);
       outputStream.close();
       outputStream = null;
+      writer = null;
       logger.trace("Summary: Wrote {} bytes to {}", writeSize, path);
       return writeSize;
     }
@@ -297,12 +291,12 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
 
   protected VectorContainer currentContainer;
   protected int pointer = 0;
-  protected final OperatorContext context;
+  protected final BufferAllocator allocator;
   protected BatchSchema schema;
 
-  public BatchGroup(VectorContainer container, OperatorContext context) {
+  public BatchGroup(VectorContainer container, BufferAllocator allocator) {
     this.currentContainer = container;
-    this.context = context;
+    this.allocator = allocator;
   }
 
   /**
@@ -311,7 +305,7 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
    * @param schema
    */
   public void setSchema(BatchSchema schema) {
-    currentContainer = SchemaUtil.coerceContainer(currentContainer, schema, context);
+    currentContainer = SchemaUtil.coerceContainer(currentContainer, schema, allocator);
     this.schema = schema;
   }
 
@@ -371,4 +365,20 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
   public SelectionVector4 getSelectionVector4() {
     throw new UnsupportedOperationException();
   }
+
+  public static void closeAll(Collection<? extends BatchGroup> groups) {
+    Exception ex = null;
+    for (BatchGroup group: groups) {
+      try {
+        group.close();
+      } catch (Exception e) {
+        ex = (ex == null) ? e : ex;
+      }
+    }
+    if (ex != null) {
+      throw UserException.dataWriteError(ex)
+          .message("Failure while flushing spilled data")
+          .build(logger);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
new file mode 100644
index 0000000..8f1b5e5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BufferedBatches.java
@@ -0,0 +1,232 @@
+/*
+ * 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.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.OperExecContext;
+import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
+import org.apache.drill.exec.physical.impl.xsort.managed.BatchGroup.InputBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.SchemaUtil;
+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.selection.SelectionVector2;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Represents the set of in-memory batches accumulated by
+ * the external sort.
+ */
+
+public class BufferedBatches {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferedBatches.class);
+
+  /**
+   * Incoming batches buffered in memory prior to spilling
+   * or an in-memory merge.
+   */
+
+  private LinkedList<BatchGroup.InputBatch> bufferedBatches = Lists.newLinkedList();
+
+  private final SorterWrapper sorterWrapper;
+
+  private BatchSchema schema;
+
+  private final OperExecContext context;
+
+  public BufferedBatches(OperExecContext opContext) {
+    context = opContext;
+    sorterWrapper = new SorterWrapper(opContext);
+  }
+
+  public void setSchema(BatchSchema schema) {
+    this.schema = schema;
+
+    // New schema: must generate a new sorter and copier.
+
+    sorterWrapper.close();
+
+    // Coerce all existing batches to the new schema.
+
+    for (BatchGroup b : bufferedBatches) {
+      b.setSchema(schema);
+    }
+  }
+
+  public int size() { return bufferedBatches.size(); }
+
+  @SuppressWarnings("resource")
+  public void add(VectorAccessible incoming, int batchSize) {
+    // Convert the incoming batch to the agreed-upon schema.
+    // No converted batch means we got an empty input batch.
+    // Converting the batch transfers memory ownership to our
+    // allocator. This gives a round-about way to learn the batch
+    // size: check the before and after memory levels, then use
+    // the difference as the batch size, in bytes.
+
+    VectorContainer convertedBatch = convertBatch(incoming);
+    if (convertedBatch == null) {
+      return;
+    }
+
+    SelectionVector2 sv2;
+    try {
+      sv2 = makeSelectionVector(incoming);
+    } catch (Exception e) {
+      convertedBatch.clear();
+      throw e;
+    }
+
+    // Sort the incoming batch using either the original selection vector,
+    // or a new one created here.
+
+    sorterWrapper.sortBatch(convertedBatch, sv2);
+    bufferBatch(convertedBatch, sv2, batchSize);
+  }
+  /**
+   * Convert an incoming batch into the agree-upon format.
+   * @param incoming
+   *
+   * @return the converted batch, or null if the incoming batch is empty
+   */
+
+  @SuppressWarnings("resource")
+  private VectorContainer convertBatch(VectorAccessible incoming) {
+
+    // Must accept the batch even if no records. Then clear
+    // the vectors to release memory since we won't do any
+    // further processing with the empty batch.
+
+    VectorContainer convertedBatch = SchemaUtil.coerceContainer(incoming, schema, context.getAllocator());
+    if (incoming.getRecordCount() == 0) {
+      for (VectorWrapper<?> w : convertedBatch) {
+        w.clear();
+      }
+      SelectionVector2 sv2 = incoming.getSelectionVector2();
+      if (sv2 != null) {
+        sv2.clear();
+      }
+      return null;
+    }
+    return convertedBatch;
+  }
+
+  private SelectionVector2 makeSelectionVector(VectorAccessible incoming) {
+    if (incoming.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE) {
+      return incoming.getSelectionVector2().clone();
+    } else {
+      return newSV2(incoming);
+    }
+  }
+
+  /**
+   * Allocate and initialize the selection vector used as the sort index.
+   * Assumes that memory is available for the vector since memory management
+   * ensured space is available.
+   *
+   * @return a new, populated selection vector 2
+   */
+
+  private SelectionVector2 newSV2(VectorAccessible incoming) {
+    SelectionVector2 sv2 = new SelectionVector2(context.getAllocator());
+    if (!sv2.allocateNewSafe(incoming.getRecordCount())) {
+      throw UserException.resourceError(new OutOfMemoryException("Unable to allocate sv2 buffer"))
+            .build(logger);
+    }
+    for (int i = 0; i < incoming.getRecordCount(); i++) {
+      sv2.setIndex(i, (char) i);
+    }
+    sv2.setRecordCount(incoming.getRecordCount());
+    return sv2;
+  }
+
+  @SuppressWarnings("resource")
+  private void bufferBatch(VectorContainer convertedBatch, SelectionVector2 sv2, int netSize) {
+    BufferAllocator allocator = context.getAllocator();
+    RecordBatchData rbd = new RecordBatchData(convertedBatch, allocator);
+    try {
+      rbd.setSv2(sv2);
+      bufferedBatches.add(new BatchGroup.InputBatch(rbd.getContainer(), rbd.getSv2(), allocator, netSize));
+
+    } catch (Throwable t) {
+      rbd.clear();
+      throw t;
+    }
+  }
+
+  public List<BatchGroup> prepareSpill(long targetSpillSize) {
+
+    // Determine the number of batches to spill to create a spill file
+    // of the desired size. The actual file size might be a bit larger
+    // or smaller than the target, which is expected.
+
+    int spillCount = 0;
+    long spillSize = 0;
+    for (InputBatch batch : bufferedBatches) {
+      long batchSize = batch.getDataSize();
+      spillSize += batchSize;
+      spillCount++;
+      if (spillSize + batchSize / 2 > targetSpillSize) {
+        break; }
+    }
+
+    // Must always spill at least 2, even if this creates an over-size
+    // spill file. But, if this is a final consolidation, we may have only
+    // a single batch.
+
+    spillCount = Math.max(spillCount, 2);
+    spillCount = Math.min(spillCount, bufferedBatches.size());
+    return SpilledRuns.prepareSpillBatches(bufferedBatches, spillCount);
+  }
+
+  public List<BatchGroup.InputBatch> removeAll() {
+    List<BatchGroup.InputBatch> batches = new ArrayList<>( );
+    batches.addAll(bufferedBatches);
+    bufferedBatches.clear();
+    return batches;
+  }
+
+  public void close() {
+    // Use the spilled runs version. In-memory batches won't throw
+    // an error, but the API is generic.
+
+    RuntimeException ex = null;
+    try {
+      BatchGroup.closeAll(bufferedBatches);
+      bufferedBatches.clear();
+    } catch (RuntimeException e) {
+      ex = e;
+    }
+    try {
+      sorterWrapper.close();
+    } catch (RuntimeException e) {
+      ex = (ex == null) ? e : ex;
+    }
+    if (ex != null) {
+      throw ex;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/90f43bff/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
deleted file mode 100644
index c6b2dd9..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/CopierHolder.java
+++ /dev/null
@@ -1,322 +0,0 @@
-/*
- * 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<? extends BatchGroup> 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<? extends BatchGroup> 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 do the actual writes. If the copier has not yet been
-   * created, generate 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
-   */
-
-  @SuppressWarnings("unchecked")
-  private void createCopier(VectorAccessible batch, List<? extends BatchGroup> batchGroupList, VectorContainer outputContainer) {
-    if (copier != null) {
-      opCodeGen.closeCopier();
-    } else {
-      copier = opCodeGen.getCopier(batch);
-    }
-
-    // Initialize the value vectors for the output container
-
-    for (VectorWrapper<?> i : batch) {
-      @SuppressWarnings("resource")
-      ValueVector v = TypeHelper.getNewVector(i.getField(), allocator);
-      outputContainer.add(v);
-    }
-    try {
-      copier.setup(context, allocator, batch, (List<BatchGroup>) batchGroupList, outputContainer);
-    } catch (SchemaChangeException e) {
-      throw UserException.unsupportedError(e)
-            .message("Unexpected schema change - likely code error.")
-            .build(logger);
-    }
-  }
-
-  public BufferAllocator getAllocator() { return allocator; }
-
-  public void close() {
-    opCodeGen.closeCopier();
-    copier = null;
-  }
-
-  /**
-   * We've gathered a set of batches, each of which has been sorted. The batches
-   * may have passed through a filter and thus may have "holes" where rows have
-   * been filtered out. We will spill records in blocks of targetRecordCount.
-   * To prepare, copy that many records into an outputContainer as a set of
-   * contiguous values in new vectors. The result is a single batch with
-   * vectors that combine a collection of input batches up to the
-   * given threshold.
-   * <p>
-   * Input. Here the top line is a selection vector of indexes.
-   * The second line is a set of batch groups (separated by underscores)
-   * with letters indicating individual records:<pre>
-   * [3 7 4 8 0 6 1] [5 3 6 8 2 0]
-   * [eh_ad_ibf]     [r_qm_kn_p]</pre>
-   * <p>
-   * Output, assuming blocks of 5 records. The brackets represent
-   * batches, the line represents the set of batches copied to the
-   * spill file.<pre>
-   * [abcde] [fhikm] [npqr]</pre>
-   * <p>
-   * The copying operation does a merge as well: copying
-   * values from the sources in ordered fashion. Consider a different example,
-   * we want to merge two input batches to produce a single output batch:
-   * <pre>
-   * Input:  [aceg] [bdfh]
-   * Output: [abcdefgh]</pre>
-   * <p>
-   * In the above, the input consists of two sorted batches. (In reality,
-   * the input batches have an associated selection vector, but that is omitted
-   * here and just the sorted values shown.) The output is a single batch
-   * with the merged records (indicated by letters) from the two input batches.
-   * <p>
-   * Here we bind the copier to the batchGroupList of sorted, buffered batches
-   * to be merged. We bind the copier output to outputContainer: the copier will write its
-   * merged "batches" of records to that container.
-   * <p>
-   * Calls to the {@link #next()} method sequentially return merged batches
-   * of the desired row count.
-    */
-
-  public static class BatchMerger implements SortResults, AutoCloseable {
-
-    private CopierHolder holder;
-    private VectorContainer hyperBatch;
-    private VectorContainer outputContainer;
-    private int targetRecordCount;
-    private int copyCount;
-    private int batchCount;
-    private long estBatchSize;
-
-    /**
-     * Creates a merger with an temporary output container.
-     *
-     * @param holder the copier that does the work
-     * @param schema schema for the input and output batches
-     * @param batchGroupList the input batches
-     * @param targetRecordCount number of records for each output batch
-     */
-    private BatchMerger(CopierHolder holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
-                        int targetRecordCount) {
-      this(holder, schema, batchGroupList, new VectorContainer(), targetRecordCount);
-    }
-
-    /**
-     * Creates a merger with the specified output container
-     *
-     * @param holder the copier that does the work
-     * @param schema schema for the input and output batches
-     * @param batchGroupList the input batches
-     * @param outputContainer merges output batch into the given output container
-     * @param targetRecordCount number of records for each output batch
-     */
-    private BatchMerger(CopierHolder holder, BatchSchema schema, List<? extends BatchGroup> batchGroupList,
-                        VectorContainer outputContainer, int targetRecordCount) {
-      this.holder = holder;
-      hyperBatch = constructHyperBatch(schema, batchGroupList);
-      copyCount = 0;
-      this.targetRecordCount = targetRecordCount;
-      this.outputContainer = outputContainer;
-      holder.createCopier(hyperBatch, batchGroupList, outputContainer);
-    }
-
-    /**
-     * Return the output container.
-     *
-     * @return the output container
-     */
-    public VectorContainer getOutput() {
-      return outputContainer;
-    }
-
-    /**
-     * Read the next merged batch. The batch holds the specified row count, but
-     * may be less if this is the last batch.
-     *
-     * @return the number of rows in the batch, or 0 if no more batches
-     * are available
-     */
-
-    @Override
-    public boolean next() {
-      Stopwatch w = Stopwatch.createStarted();
-      long start = holder.allocator.getAllocatedMemory();
-      int count = holder.copier.next(targetRecordCount);
-      copyCount += count;
-      if (count > 0) {
-        long t = w.elapsed(TimeUnit.MICROSECONDS);
-        batchCount++;
-        logger.trace("Took {} us to merge {} records", t, count);
-        long size = holder.allocator.getAllocatedMemory() - start;
-        estBatchSize = Math.max(estBatchSize, size);
-      } else {
-        logger.trace("copier returned 0 records");
-      }
-
-      // Identify the schema to be used in the output container. (Since
-      // all merged batches have the same schema, the schema we identify
-      // here should be the same as that which we already had.
-
-      outputContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-
-      // The copier does not set the record count in the output
-      // container, so do that here.
-
-      outputContainer.setRecordCount(count);
-
-      return count > 0;
-    }
-
-    /**
-     * Construct a vector container that holds a list of batches, each represented as an
-     * array of vectors. The entire collection of vectors has a common schema.
-     * <p>
-     * To build the collection, we go through the current schema (which has been
-     * devised to be common for all batches.) For each field in the schema, we create
-     * an array of vectors. To create the elements, we iterate over all the incoming
-     * batches and search for the vector that matches the current column.
-     * <p>
-     * Finally, we build a new schema for the combined container. That new schema must,
-     * because of the way the container was created, match the current schema.
-     *
-     * @param schema schema for the hyper batch
-     * @param batchGroupList list of batches to combine
-     * @return a container where each column is represented as an array of vectors
-     * (hence the "hyper" in the method name)
-     */
-
-    private VectorContainer constructHyperBatch(BatchSchema schema, List<? extends BatchGroup> batchGroupList) {
-      VectorContainer cont = new VectorContainer();
-      for (MaterializedField field : schema) {
-        ValueVector[] vectors = new ValueVector[batchGroupList.size()];
-        int i = 0;
-        for (BatchGroup group : batchGroupList) {
-          vectors[i++] = group.getValueAccessorById(
-              field.getValueClass(),
-              group.getValueVectorId(SchemaPath.getSimplePath(field.getPath())).getFieldIds())
-              .getValueVector();
-        }
-        cont.add(vectors);
-      }
-      cont.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);
-      return cont;
-    }
-
-    @Override
-    public void close() {
-      hyperBatch.clear();
-      holder.close();
-    }
-
-    @Override
-    public int getRecordCount() {
-      return copyCount;
-    }
-
-    @Override
-    public int getBatchCount() {
-      return batchCount;
-    }
-
-    /**
-     * Gets the estimated batch size, in bytes. Use for estimating the memory
-     * needed to process the batches that this operator created.
-     * @return the size of the largest batch created by this operation,
-     * in bytes
-     */
-
-    public long getEstBatchSize() {
-      return estBatchSize;
-    }
-  }
-}