[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-17 Thread paul-rogers
Github user paul-rogers closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618560
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleSetImpl.java
 ---
@@ -0,0 +1,551 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import 
org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.VectorContainerBuilder;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VectorOverflowException;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
+import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
+
+/**
+ * Implementation of a column when creating a row batch.
+ * Every column resides at an index, is defined by a schema,
+ * is backed by a value vector, and and is written to by a writer.
+ * Each column also tracks the schema version in which it was added
+ * to detect schema evolution. Each column has an optional overflow
+ * vector that holds overflow record values when a batch becomes
+ * full.
+ * 
+ * Overflow vectors require special consideration. The vector class itself
+ * must remain constant as it is bound to the writer. To handle overflow,
+ * the implementation must replace the buffer in the vector with a new
+ * one, saving the full vector to return as part of the final row batch.
+ * This puts the column in one of three states:
+ * 
+ * Normal: only one vector is of concern - the vector for the active
+ * row batch.
+ * Overflow: a write to a vector caused overflow. For all columns,
+ * the data buffer is shifted to a harvested vector, and a new, empty
+ * buffer is put into the active vector.
+ * Excess: a (small) column received values for the row that will
--- End diff --

The term "excess" is perhaps not as descriptive as we'd like...

Consider a row with three columns: (a, b, c).

We're on row 12,345. We write a value for column a. We then try to write b. 
But, b is a large VarChar and so were the previous b values. b overflows. We 
will write the value for b and (later) c to the new, look-ahead vectors. But, 
we also have to copy the "excess" value for a from the original a vector to the 
new look-ahead vector for a.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618019
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/LogicalTupleLoader.java
 ---
@@ -0,0 +1,204 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import org.apache.drill.exec.physical.rowSet.TupleSchema.TupleColumnSchema;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Shim inserted between an actual tuple loader and the client to remove 
columns
+ * that are not projected from input to output. The underlying loader 
handles only
+ * the projected columns in order to improve efficiency. This class 
presents the
+ * full table schema, but returns null for the non-projected columns. This 
allows
+ * the reader to work with the table schema as defined by the data source, 
but
+ * skip those columns which are not projected. Skipping non-projected 
columns avoids
+ * creating value vectors which are immediately discarded. It may also 
save the reader
+ * from reading unwanted data.
+ */
+public class LogicalTupleLoader implements TupleLoader {
+
+  public static final int UNMAPPED = -1;
+
+  private static class MappedColumn implements TupleColumnSchema {
+
+private final MaterializedField schema;
+private final int mapping;
+
+public MappedColumn(MaterializedField schema, int mapping) {
+  this.schema = schema;
+  this.mapping = mapping;
+}
+
+@Override
+public MaterializedField schema() { return schema; }
+
+@Override
+public boolean isSelected() { return mapping != UNMAPPED; }
+
+@Override
+public int vectorIndex() { return mapping; }
+  }
+
+  /**
+   * Implementation of the tuple schema that describes the full data source
+   * schema. The underlying loader schema is a subset of these columns. 
Note
+   * that the columns appear in the same order in both schemas, but the 
loader
+   * schema is a subset of the table schema.
+   */
+
+  private class LogicalTupleSchema implements TupleSchema {
+
+private final Set selection = new HashSet<>();
+private final TupleSchema physicalSchema;
+
+private LogicalTupleSchema(TupleSchema physicalSchema, 
Collection selection) {
+  this.physicalSchema = physicalSchema;
+  this.selection.addAll(selection);
+}
+
+@Override
+public int columnCount() { return logicalSchema.count(); }
+
+@Override
+public int columnIndex(String colName) {
+  return logicalSchema.indexOf(rsLoader.toKey(colName));
+}
+
+@Override
+public TupleColumnSchema metadata(int colIndex) { return 
logicalSchema.get(colIndex); }
+
+@Override
+public MaterializedField column(int colIndex) { return 
logicalSchema.get(colIndex).schema(); }
+
+@Override
+public TupleColumnSchema metadata(String colName) { return 
logicalSchema.get(colName); }
+
+@Override
+public MaterializedField column(String colName) { return 
logicalSchema.get(colName).schema(); }
+
+@Override
+public int addColumn(MaterializedField columnSchema) {
+  String key = rsLoader.toKey(columnSchema.getName());
+  int pIndex;
+  if (selection.contains(key)) {
--- End diff --

Removed this feature. Now use a case-insensitive map for 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618655
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleSetImpl.java
 ---
@@ -0,0 +1,551 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import 
org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.VectorContainerBuilder;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VectorOverflowException;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
+import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
+
+/**
+ * Implementation of a column when creating a row batch.
+ * Every column resides at an index, is defined by a schema,
+ * is backed by a value vector, and and is written to by a writer.
+ * Each column also tracks the schema version in which it was added
+ * to detect schema evolution. Each column has an optional overflow
+ * vector that holds overflow record values when a batch becomes
+ * full.
+ * 
+ * Overflow vectors require special consideration. The vector class itself
+ * must remain constant as it is bound to the writer. To handle overflow,
+ * the implementation must replace the buffer in the vector with a new
+ * one, saving the full vector to return as part of the final row batch.
+ * This puts the column in one of three states:
+ * 
+ * Normal: only one vector is of concern - the vector for the active
+ * row batch.
+ * Overflow: a write to a vector caused overflow. For all columns,
+ * the data buffer is shifted to a harvested vector, and a new, empty
+ * buffer is put into the active vector.
+ * Excess: a (small) column received values for the row that will
+ * overflow due to a later column. When overflow occurs, the excess
+ * column value, from the overflow record, resides in the active
+ * vector. It must be shifted from the active vector into the new
+ * overflow buffer.
+ */
+
+public class TupleSetImpl implements TupleSchema {
+
+  public static class TupleLoaderImpl implements TupleLoader {
+
+public TupleSetImpl tupleSet;
+
+public TupleLoaderImpl(TupleSetImpl tupleSet) {
+  this.tupleSet = tupleSet;
+}
+
+@Override
+public TupleSchema schema() { return tupleSet; }
+
+@Override
+public ColumnLoader column(int colIndex) {
+  // TODO: Cache loaders here
+  return tupleSet.columnImpl(colIndex).writer;
+}
+
+@Override
+public ColumnLoader column(String colName) {
+  ColumnImpl col = tupleSet.columnImpl(colName);
+  if (col == null) {
+throw new UndefinedColumnException(colName);
+  }
+  return col.writer;
+}
+
+@Override
+public TupleLoader loadRow(Object... values) {
--- End diff --

Type validation is done as the values wend their way through the writer 
tree. Eventually, we'll notice that the value is, say, Integer and call the 
`setInt()` method on the writer. If the writer is really for, say, a VarChar, 
then an unsupported operation exception will be thrown at that time. Similarly, 
if the object type is not one we know how to parse, then an exception will be 
thrown at that 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618125
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
 ---
@@ -0,0 +1,412 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Implementation of the result set loader.
+ * @see {@link ResultSetLoader}
+ */
+
+public class ResultSetLoaderImpl implements ResultSetLoader, 
WriterIndexImpl.WriterIndexListener {
+
+  public static class ResultSetOptions {
+public final int vectorSizeLimit;
+public final int rowCountLimit;
+public final boolean caseSensitive;
+public final ResultVectorCache inventory;
+private final Collection selection;
+
+public ResultSetOptions() {
+  vectorSizeLimit = ValueVector.MAX_BUFFER_SIZE;
+  rowCountLimit = ValueVector.MAX_ROW_COUNT;
+  caseSensitive = false;
+  selection = null;
+  inventory = null;
+}
+
+public ResultSetOptions(OptionBuilder builder) {
+  this.vectorSizeLimit = builder.vectorSizeLimit;
+  this.rowCountLimit = builder.rowCountLimit;
+  this.caseSensitive = builder.caseSensitive;
+  this.selection = builder.selection;
+  this.inventory = builder.inventory;
+}
+  }
+
+  public static class OptionBuilder {
+private int vectorSizeLimit;
+private int rowCountLimit;
+private boolean caseSensitive;
+private Collection selection;
+private ResultVectorCache inventory;
+
+public OptionBuilder() {
+  ResultSetOptions options = new ResultSetOptions();
+  vectorSizeLimit = options.vectorSizeLimit;
+  rowCountLimit = options.rowCountLimit;
+  caseSensitive = options.caseSensitive;
+}
+
+public OptionBuilder setCaseSensitive(boolean flag) {
+  caseSensitive = flag;
+  return this;
+}
+
+public OptionBuilder setRowCountLimit(int limit) {
+  rowCountLimit = Math.min(limit, ValueVector.MAX_ROW_COUNT);
+  return this;
+}
+
+public OptionBuilder setSelection(Collection selection) {
+  this.selection = selection;
+  return this;
+}
+
+public OptionBuilder setVectorCache(ResultVectorCache inventory) {
+  this.inventory = inventory;
+  return this;
+}
+
+// TODO: No setter for vector length yet: is hard-coded
+// at present in the value vector.
+
+public ResultSetOptions build() {
+  return new ResultSetOptions(this);
+}
+  }
+
+  public static class VectorContainerBuilder {
+private final ResultSetLoaderImpl rowSetMutator;
+private int lastUpdateVersion = -1;
+private VectorContainer container;
+
+public VectorContainerBuilder(ResultSetLoaderImpl rowSetMutator) {
+  this.rowSetMutator = rowSetMutator;
+  container = new VectorContainer(rowSetMutator.allocator);
+}
+
+public void update() {
+  if (lastUpdateVersion < rowSetMutator.schemaVersion()) {
+rowSetMutator.rootTuple.buildContainer(this);
+container.buildSchema(SelectionVectorMode.NONE);
+lastUpdateVersion = rowSetMutator.schemaVersion();
+  }
+}
+
+public VectorContainer container() { return container; }
+
+public int 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133617837
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/TupleSchema.java
 ---
@@ -0,0 +1,91 @@
+/*
+ * 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.rowSet;
+
+import org.apache.drill.exec.physical.rowSet.impl.MaterializedSchema;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Defines the schema of a tuple: either the top-level row or a nested
+ * "map" (really structure). A schema is a collection of columns (backed
+ * by vectors in the loader itself.) Columns are accessible by name or
+ * index. New columns may be added at any time; the new column takes the
+ * next available index.
+ */
+
+public interface TupleSchema {
+
+  public interface TupleColumnSchema {
+MaterializedField schema();
+
+/**
+ * Report if a column is selected.
+ * @param colIndex index of the column to check
+ * @return true if the column is selected (data is collected),
+ * false if the column is unselected (data is discarded)
+ */
+
+boolean isSelected();
--- End diff --

Changed. Should be "isProjected". This means that the column (that, in this 
case, exists in a table), is projected into the result set. Assume a table has 
the schema

(a, b, c)

But the query is:

```SELECT c, a FROM ...```

Then a and c are "projected", b is unprojected.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133617874
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnLoaderImpl.java
 ---
@@ -0,0 +1,31 @@
+/*
+ * 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.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+
+/**
+ * Implementation interface for a column loader. Adds to the public 
interface
+ * a number of methods needed to coordinate batch overflow.
+ */
+
+public interface ColumnLoaderImpl extends ColumnLoader {
--- End diff --

Agreed. Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133617740
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
 ---
@@ -0,0 +1,170 @@
+/*
+ * 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.rowSet;
+
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Builds a result set (series of zero or more row sets) based on a defined
+ * schema which may
+ * evolve (expand) over time. Automatically rolls "overflow" rows over
+ * when a batch fills.
+ * 
+ * Many of the methods in this interface are verify that the loader is
--- End diff --

Fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618354
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCache.java
 ---
@@ -0,0 +1,181 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Manages an inventory of value vectors used across row batch readers.
+ * Drill semantics for batches is complex. Each operator logically returns
+ * a batch of records on each call of the Drill Volcano iterator protocol
+ * next() operation. However, the batches "returned" are not
+ * separate objects. Instead, Drill enforces the following semantics:
+ * 
+ * If a next() call returns OK then the set of 
vectors
+ * in the "returned" batch must be identical to those in the prior batch. 
Not
+ * just the same type; they must be the same ValueVector objects.
+ * (The buffers within the vectors will be different.)
+ * If the set of vectors changes in any way (add a vector, remove a
+ * vector, change the type of a vector), then the next() call
+ * must return OK_NEW_SCHEMA.
+ * 
+ * These rules create interesting constraints for the scan operator.
+ * Conceptually, each batch is distinct. But, it must share vectors. The
+ * {@link ResultSetLoader} class handles this by managing the set of 
vectors
+ * used by a single reader.
+ * 
+ * Readers are independent: each may read a distinct schema (as in JSON.)
+ * Yet, the Drill protocol requires minimizing spurious 
OK_NEW_SCHEMA
+ * events. As a result, two readers run by the same scan operator must
+ * share the same set of vectors, despite the fact that they may have
+ * different schemas and thus different ResultSetLoaders.
+ * 
+ * The purpose of this inventory is to persist vectors across readers, even
+ * when, say, reader B does not use a vector that reader A created.
+ * 
+ * The semantics supported by this class include:
+ * 
+ * Ability to "pre-declare" columns based on columns that appear in
+ * an explicit select list. This ensures that the columns are known (but
+ * not their types).
+ * Ability to reuse a vector across readers if the column retains the 
same
+ * name and type (minor type and mode.)
+ * Ability to flush unused vectors for readers with changing schemas
+ * if a schema change occurs.
+ * Support schema "hysteresis"; that is, the a "sticky" schema that
+ * minimizes spurious changes. Once a vector is declared, it can be 
included
+ * in all subsequent batches (provided the column is nullable or an 
array.)
+ * 
+ */
+public class ResultVectorCache {
+
+  /**
+   * State of a projected vector. At first all we have is a name.
+   * Later, we'll discover the type.
+   */
+
+  private static class VectorState {
+protected final String name;
+protected ValueVector vector;
+protected boolean touched;
+
+public VectorState(String name) {
+  this.name = name;
+}
+
+public boolean satisfies(MaterializedField colSchema) {
+  if (vector == null) {
+return false;
+  }
+  MaterializedField vectorSchema = vector.getField();
+  return vectorSchema.getType().equals(colSchema.getType());
+}
+  }
+
+  private final BufferAllocator allocator;
+  private final Map vectors = new HashMap<>();
+
+  public ResultVectorCache(BufferAllocator allocator) {
+this.allocator = 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-16 Thread paul-rogers
Github user paul-rogers commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r133618058
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
 ---
@@ -0,0 +1,412 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Implementation of the result set loader.
+ * @see {@link ResultSetLoader}
+ */
+
+public class ResultSetLoaderImpl implements ResultSetLoader, 
WriterIndexImpl.WriterIndexListener {
+
+  public static class ResultSetOptions {
+public final int vectorSizeLimit;
+public final int rowCountLimit;
+public final boolean caseSensitive;
+public final ResultVectorCache inventory;
--- End diff --

Renamed to vectorCache.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r130429208
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/TupleSchema.java
 ---
@@ -0,0 +1,91 @@
+/*
+ * 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.rowSet;
+
+import org.apache.drill.exec.physical.rowSet.impl.MaterializedSchema;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Defines the schema of a tuple: either the top-level row or a nested
+ * "map" (really structure). A schema is a collection of columns (backed
+ * by vectors in the loader itself.) Columns are accessible by name or
+ * index. New columns may be added at any time; the new column takes the
+ * next available index.
+ */
+
+public interface TupleSchema {
+
+  public interface TupleColumnSchema {
+MaterializedField schema();
+
+/**
+ * Report if a column is selected.
+ * @param colIndex index of the column to check
+ * @return true if the column is selected (data is collected),
+ * false if the column is unselected (data is discarded)
+ */
+
+boolean isSelected();
--- End diff --

What does it mean for a  column to be selected? Selected in the query?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131564509
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCache.java
 ---
@@ -0,0 +1,181 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Manages an inventory of value vectors used across row batch readers.
+ * Drill semantics for batches is complex. Each operator logically returns
+ * a batch of records on each call of the Drill Volcano iterator protocol
+ * next() operation. However, the batches "returned" are not
+ * separate objects. Instead, Drill enforces the following semantics:
+ * 
+ * If a next() call returns OK then the set of 
vectors
+ * in the "returned" batch must be identical to those in the prior batch. 
Not
+ * just the same type; they must be the same ValueVector objects.
+ * (The buffers within the vectors will be different.)
+ * If the set of vectors changes in any way (add a vector, remove a
+ * vector, change the type of a vector), then the next() call
+ * must return OK_NEW_SCHEMA.
+ * 
+ * These rules create interesting constraints for the scan operator.
+ * Conceptually, each batch is distinct. But, it must share vectors. The
+ * {@link ResultSetLoader} class handles this by managing the set of 
vectors
+ * used by a single reader.
+ * 
+ * Readers are independent: each may read a distinct schema (as in JSON.)
+ * Yet, the Drill protocol requires minimizing spurious 
OK_NEW_SCHEMA
+ * events. As a result, two readers run by the same scan operator must
+ * share the same set of vectors, despite the fact that they may have
+ * different schemas and thus different ResultSetLoaders.
+ * 
+ * The purpose of this inventory is to persist vectors across readers, even
+ * when, say, reader B does not use a vector that reader A created.
+ * 
+ * The semantics supported by this class include:
+ * 
+ * Ability to "pre-declare" columns based on columns that appear in
+ * an explicit select list. This ensures that the columns are known (but
+ * not their types).
+ * Ability to reuse a vector across readers if the column retains the 
same
+ * name and type (minor type and mode.)
+ * Ability to flush unused vectors for readers with changing schemas
+ * if a schema change occurs.
+ * Support schema "hysteresis"; that is, the a "sticky" schema that
+ * minimizes spurious changes. Once a vector is declared, it can be 
included
+ * in all subsequent batches (provided the column is nullable or an 
array.)
+ * 
+ */
+public class ResultVectorCache {
+
+  /**
+   * State of a projected vector. At first all we have is a name.
+   * Later, we'll discover the type.
+   */
+
+  private static class VectorState {
+protected final String name;
+protected ValueVector vector;
+protected boolean touched;
+
+public VectorState(String name) {
+  this.name = name;
+}
+
+public boolean satisfies(MaterializedField colSchema) {
+  if (vector == null) {
+return false;
+  }
+  MaterializedField vectorSchema = vector.getField();
+  return vectorSchema.getType().equals(colSchema.getType());
+}
+  }
+
+  private final BufferAllocator allocator;
+  private final Map vectors = new HashMap<>();
+
+  public ResultVectorCache(BufferAllocator allocator) {
+this.allocator = 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131284158
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/LogicalTupleLoader.java
 ---
@@ -0,0 +1,204 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import org.apache.drill.exec.physical.rowSet.TupleSchema.TupleColumnSchema;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Shim inserted between an actual tuple loader and the client to remove 
columns
+ * that are not projected from input to output. The underlying loader 
handles only
+ * the projected columns in order to improve efficiency. This class 
presents the
+ * full table schema, but returns null for the non-projected columns. This 
allows
+ * the reader to work with the table schema as defined by the data source, 
but
+ * skip those columns which are not projected. Skipping non-projected 
columns avoids
+ * creating value vectors which are immediately discarded. It may also 
save the reader
+ * from reading unwanted data.
+ */
+public class LogicalTupleLoader implements TupleLoader {
+
+  public static final int UNMAPPED = -1;
+
+  private static class MappedColumn implements TupleColumnSchema {
+
+private final MaterializedField schema;
+private final int mapping;
+
+public MappedColumn(MaterializedField schema, int mapping) {
+  this.schema = schema;
+  this.mapping = mapping;
+}
+
+@Override
+public MaterializedField schema() { return schema; }
+
+@Override
+public boolean isSelected() { return mapping != UNMAPPED; }
+
+@Override
+public int vectorIndex() { return mapping; }
+  }
+
+  /**
+   * Implementation of the tuple schema that describes the full data source
+   * schema. The underlying loader schema is a subset of these columns. 
Note
+   * that the columns appear in the same order in both schemas, but the 
loader
+   * schema is a subset of the table schema.
+   */
+
+  private class LogicalTupleSchema implements TupleSchema {
+
+private final Set selection = new HashSet<>();
+private final TupleSchema physicalSchema;
+
+private LogicalTupleSchema(TupleSchema physicalSchema, 
Collection selection) {
+  this.physicalSchema = physicalSchema;
+  this.selection.addAll(selection);
+}
+
+@Override
+public int columnCount() { return logicalSchema.count(); }
+
+@Override
+public int columnIndex(String colName) {
+  return logicalSchema.indexOf(rsLoader.toKey(colName));
+}
+
+@Override
+public TupleColumnSchema metadata(int colIndex) { return 
logicalSchema.get(colIndex); }
+
+@Override
+public MaterializedField column(int colIndex) { return 
logicalSchema.get(colIndex).schema(); }
+
+@Override
+public TupleColumnSchema metadata(String colName) { return 
logicalSchema.get(colName); }
+
+@Override
+public MaterializedField column(String colName) { return 
logicalSchema.get(colName).schema(); }
+
+@Override
+public int addColumn(MaterializedField columnSchema) {
+  String key = rsLoader.toKey(columnSchema.getName());
+  int pIndex;
+  if (selection.contains(key)) {
--- End diff --

selection is already normalized if caseSensitive is false ?

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131216670
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnLoaderImpl.java
 ---
@@ -0,0 +1,31 @@
+/*
+ * 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.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+
+/**
+ * Implementation interface for a column loader. Adds to the public 
interface
+ * a number of methods needed to coordinate batch overflow.
+ */
+
+public interface ColumnLoaderImpl extends ColumnLoader {
--- End diff --

"Impl"  in an interface name sounds odd.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r130250994
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
 ---
@@ -0,0 +1,170 @@
+/*
+ * 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.rowSet;
+
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Builds a result set (series of zero or more row sets) based on a defined
+ * schema which may
+ * evolve (expand) over time. Automatically rolls "overflow" rows over
+ * when a batch fills.
+ * 
+ * Many of the methods in this interface are verify that the loader is
--- End diff --

"to verify"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131554894
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
 ---
@@ -0,0 +1,412 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Implementation of the result set loader.
+ * @see {@link ResultSetLoader}
+ */
+
+public class ResultSetLoaderImpl implements ResultSetLoader, 
WriterIndexImpl.WriterIndexListener {
+
+  public static class ResultSetOptions {
+public final int vectorSizeLimit;
+public final int rowCountLimit;
+public final boolean caseSensitive;
+public final ResultVectorCache inventory;
+private final Collection selection;
+
+public ResultSetOptions() {
+  vectorSizeLimit = ValueVector.MAX_BUFFER_SIZE;
+  rowCountLimit = ValueVector.MAX_ROW_COUNT;
+  caseSensitive = false;
+  selection = null;
+  inventory = null;
+}
+
+public ResultSetOptions(OptionBuilder builder) {
+  this.vectorSizeLimit = builder.vectorSizeLimit;
+  this.rowCountLimit = builder.rowCountLimit;
+  this.caseSensitive = builder.caseSensitive;
+  this.selection = builder.selection;
+  this.inventory = builder.inventory;
+}
+  }
+
+  public static class OptionBuilder {
+private int vectorSizeLimit;
+private int rowCountLimit;
+private boolean caseSensitive;
+private Collection selection;
+private ResultVectorCache inventory;
+
+public OptionBuilder() {
+  ResultSetOptions options = new ResultSetOptions();
+  vectorSizeLimit = options.vectorSizeLimit;
+  rowCountLimit = options.rowCountLimit;
+  caseSensitive = options.caseSensitive;
+}
+
+public OptionBuilder setCaseSensitive(boolean flag) {
+  caseSensitive = flag;
+  return this;
+}
+
+public OptionBuilder setRowCountLimit(int limit) {
+  rowCountLimit = Math.min(limit, ValueVector.MAX_ROW_COUNT);
+  return this;
+}
+
+public OptionBuilder setSelection(Collection selection) {
+  this.selection = selection;
+  return this;
+}
+
+public OptionBuilder setVectorCache(ResultVectorCache inventory) {
+  this.inventory = inventory;
+  return this;
+}
+
+// TODO: No setter for vector length yet: is hard-coded
+// at present in the value vector.
+
+public ResultSetOptions build() {
+  return new ResultSetOptions(this);
+}
+  }
+
+  public static class VectorContainerBuilder {
+private final ResultSetLoaderImpl rowSetMutator;
+private int lastUpdateVersion = -1;
+private VectorContainer container;
+
+public VectorContainerBuilder(ResultSetLoaderImpl rowSetMutator) {
+  this.rowSetMutator = rowSetMutator;
+  container = new VectorContainer(rowSetMutator.allocator);
+}
+
+public void update() {
+  if (lastUpdateVersion < rowSetMutator.schemaVersion()) {
+rowSetMutator.rootTuple.buildContainer(this);
+container.buildSchema(SelectionVectorMode.NONE);
+lastUpdateVersion = rowSetMutator.schemaVersion();
+  }
+}
+
+public VectorContainer container() { return container; }
+
+public int 

[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131459203
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
 ---
@@ -0,0 +1,412 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Implementation of the result set loader.
+ * @see {@link ResultSetLoader}
+ */
+
+public class ResultSetLoaderImpl implements ResultSetLoader, 
WriterIndexImpl.WriterIndexListener {
+
+  public static class ResultSetOptions {
+public final int vectorSizeLimit;
+public final int rowCountLimit;
+public final boolean caseSensitive;
+public final ResultVectorCache inventory;
--- End diff --

The name 'inventory' does not convey the intent clearly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131684173
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleSetImpl.java
 ---
@@ -0,0 +1,551 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import 
org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.VectorContainerBuilder;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VectorOverflowException;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
+import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
+
+/**
+ * Implementation of a column when creating a row batch.
+ * Every column resides at an index, is defined by a schema,
+ * is backed by a value vector, and and is written to by a writer.
+ * Each column also tracks the schema version in which it was added
+ * to detect schema evolution. Each column has an optional overflow
+ * vector that holds overflow record values when a batch becomes
+ * full.
+ * 
+ * Overflow vectors require special consideration. The vector class itself
+ * must remain constant as it is bound to the writer. To handle overflow,
+ * the implementation must replace the buffer in the vector with a new
+ * one, saving the full vector to return as part of the final row batch.
+ * This puts the column in one of three states:
+ * 
+ * Normal: only one vector is of concern - the vector for the active
+ * row batch.
+ * Overflow: a write to a vector caused overflow. For all columns,
+ * the data buffer is shifted to a harvested vector, and a new, empty
+ * buffer is put into the active vector.
+ * Excess: a (small) column received values for the row that will
+ * overflow due to a later column. When overflow occurs, the excess
+ * column value, from the overflow record, resides in the active
+ * vector. It must be shifted from the active vector into the new
+ * overflow buffer.
+ */
+
+public class TupleSetImpl implements TupleSchema {
+
+  public static class TupleLoaderImpl implements TupleLoader {
+
+public TupleSetImpl tupleSet;
+
+public TupleLoaderImpl(TupleSetImpl tupleSet) {
+  this.tupleSet = tupleSet;
+}
+
+@Override
+public TupleSchema schema() { return tupleSet; }
+
+@Override
+public ColumnLoader column(int colIndex) {
+  // TODO: Cache loaders here
+  return tupleSet.columnImpl(colIndex).writer;
+}
+
+@Override
+public ColumnLoader column(String colName) {
+  ColumnImpl col = tupleSet.columnImpl(colName);
+  if (col == null) {
+throw new UndefinedColumnException(colName);
+  }
+  return col.writer;
+}
+
+@Override
+public TupleLoader loadRow(Object... values) {
--- End diff --

Is there a need to verify the types of the incoming args?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: DRILL-5657: Implement size-aware result set loader

2017-08-08 Thread bitblender
Github user bitblender commented on a diff in the pull request:

https://github.com/apache/drill/pull/866#discussion_r131685349
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleSetImpl.java
 ---
@@ -0,0 +1,551 @@
+/*
+ * 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.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.physical.rowSet.ColumnLoader;
+import org.apache.drill.exec.physical.rowSet.TupleLoader;
+import org.apache.drill.exec.physical.rowSet.TupleSchema;
+import 
org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.VectorContainerBuilder;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VectorOverflowException;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
+import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
+
+/**
+ * Implementation of a column when creating a row batch.
+ * Every column resides at an index, is defined by a schema,
+ * is backed by a value vector, and and is written to by a writer.
+ * Each column also tracks the schema version in which it was added
+ * to detect schema evolution. Each column has an optional overflow
+ * vector that holds overflow record values when a batch becomes
+ * full.
+ * 
+ * Overflow vectors require special consideration. The vector class itself
+ * must remain constant as it is bound to the writer. To handle overflow,
+ * the implementation must replace the buffer in the vector with a new
+ * one, saving the full vector to return as part of the final row batch.
+ * This puts the column in one of three states:
+ * 
+ * Normal: only one vector is of concern - the vector for the active
+ * row batch.
+ * Overflow: a write to a vector caused overflow. For all columns,
+ * the data buffer is shifted to a harvested vector, and a new, empty
+ * buffer is put into the active vector.
+ * Excess: a (small) column received values for the row that will
--- End diff --

'Excess' is the LOOK_AHEAD state, correct? I think it would be better if 
the comments use the same terminology as in the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #866: Drill 5657: Implement size-aware result set loader

2017-07-03 Thread paul-rogers
GitHub user paul-rogers opened a pull request:

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

Drill 5657: Implement size-aware result set loader



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

$ git pull https://github.com/paul-rogers/drill DRILL-5657

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

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

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

This closes #866


commit 2c95326e1108e9299d6742c0cf5c35b86605bbe6
Author: Paul Rogers 
Date:   2017-07-04T05:43:40Z

DRILL-5657: Revised column accessors

commit bb4a4ed8eadbaa339570e8adabc8709e3d393719
Author: Paul Rogers 
Date:   2017-07-04T05:45:43Z

Revisions to row set test utilities

commit bec870fa849fdc39a12a1d1d48d18f3a0c6e8371
Author: Paul Rogers 
Date:   2017-07-04T05:48:14Z

Result set loader implementation and tests

commit 0c8727f5adfa1ffd2e4be584ba7f967e72265661
Author: Paul Rogers 
Date:   2017-07-04T05:48:47Z

Collateral damage: other affected files




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---