You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2015/02/12 22:36:47 UTC

drill git commit: DRILL-1605: ensure GenericAccessor returns null for non-existent records

Repository: drill
Updated Branches:
  refs/heads/master de89f36d9 -> aa2caa8af


DRILL-1605: ensure GenericAccessor returns null for non-existent records


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

Branch: refs/heads/master
Commit: aa2caa8af5f98b1f9fa585443714824d0fa00a12
Parents: de89f36
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Thu Feb 5 13:24:13 2015 -0800
Committer: Hanifi Gunes <hg...@maprtech.com>
Committed: Thu Feb 12 13:32:43 2015 -0800

----------------------------------------------------------------------
 .../vector/accessor/BoundCheckingAccessor.java  | 141 +++++++++++++++++++
 .../java/org/apache/drill/TestSchemaChange.java |  43 ++++++
 .../resources/schemachange/multi/first.json     |   2 +
 .../resources/schemachange/multi/second.json    |   1 +
 .../apache/drill/jdbc/DrillAccessorList.java    |   5 +-
 5 files changed, 191 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/aa2caa8a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/BoundCheckingAccessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/BoundCheckingAccessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/BoundCheckingAccessor.java
new file mode 100644
index 0000000..7e8da2c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/BoundCheckingAccessor.java
@@ -0,0 +1,141 @@
+/**
+ * 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.vector.accessor;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * A decorating accessor that returns null for indices that is beyond underlying vector's capacity.
+ */
+public class BoundCheckingAccessor implements SqlAccessor {
+  private final ValueVector vector;
+  private final SqlAccessor delegate;
+
+  public BoundCheckingAccessor(ValueVector vector, SqlAccessor inner) {
+    this.vector = vector;
+    this.delegate = inner;
+  }
+
+  @Override
+  public boolean isNull(int index) {
+    return delegate.isNull(index);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getBigDecimal(index);
+  }
+
+  @Override
+  public boolean getBoolean(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getBoolean(index);
+  }
+
+  @Override
+  public byte getByte(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getByte(index);
+  }
+
+  @Override
+  public byte[] getBytes(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getBytes(index);
+  }
+
+  @Override
+  public Date getDate(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getDate(index);
+  }
+
+  @Override
+  public double getDouble(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getDouble(index);
+  }
+
+  @Override
+  public float getFloat(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getFloat(index);
+  }
+
+  @Override
+  public char getChar(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getChar(index);
+  }
+
+  @Override
+  public int getInt(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getInt(index);
+  }
+
+  @Override
+  public long getLong(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getLong(index);
+  }
+
+  @Override
+  public short getShort(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getShort(index);
+  }
+
+  @Override
+  public InputStream getStream(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getStream(index);
+  }
+
+  @Override
+  public Reader getReader(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getReader(index);
+  }
+
+  @Override
+  public String getString(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getString(index);
+  }
+
+  @Override
+  public Time getTime(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getTime(index);
+  }
+
+  @Override
+  public Timestamp getTimestamp(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    return delegate.getTimestamp(index);
+  }
+
+  /**
+   * Returns an instance sitting at the given index if exists, null otherwise.
+   *
+   * @see org.apache.drill.exec.vector.accessor.SqlAccessor#getObject(int)
+   */
+  @Override
+  public Object getObject(int index) throws AbstractSqlAccessor.InvalidAccessException {
+    // in case some vectors have less values than others, callee invokes this method with index >= #getValueCount
+    // this should still yield null.
+    final ValueVector.Accessor accessor = vector.getAccessor();
+    if (index < accessor.getValueCount()) {
+      return delegate.getObject(index);
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/aa2caa8a/exec/java-exec/src/test/java/org/apache/drill/TestSchemaChange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestSchemaChange.java b/exec/java-exec/src/test/java/org/apache/drill/TestSchemaChange.java
new file mode 100644
index 0000000..4465e79
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestSchemaChange.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill;
+
+import org.apache.drill.common.util.TestTools;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestSchemaChange extends BaseTestQuery {
+
+  protected static final String WORKING_PATH = TestTools.getWorkingPath();
+  protected static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
+
+  @Test //DRILL-1605
+  @Ignore("Until DRILL-2171 is fixed")
+  public void testMultiFilesWithDifferentSchema() throws Exception {
+    final String query = String.format("select a, b from dfs_test.`%s/schemachange/multi/*.json`", TEST_RES_PATH);
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("a", "b")
+        .baselineValues(1L, null)
+        .baselineValues(2L, null)
+        .baselineValues(null, true)
+        .build()
+        .run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/aa2caa8a/exec/java-exec/src/test/resources/schemachange/multi/first.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/schemachange/multi/first.json b/exec/java-exec/src/test/resources/schemachange/multi/first.json
new file mode 100644
index 0000000..1c649ab
--- /dev/null
+++ b/exec/java-exec/src/test/resources/schemachange/multi/first.json
@@ -0,0 +1,2 @@
+{"id":1, "a":1}
+{"id":2, "a":2}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/aa2caa8a/exec/java-exec/src/test/resources/schemachange/multi/second.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/schemachange/multi/second.json b/exec/java-exec/src/test/resources/schemachange/multi/second.json
new file mode 100644
index 0000000..5d7d373
--- /dev/null
+++ b/exec/java-exec/src/test/resources/schemachange/multi/second.json
@@ -0,0 +1 @@
+{"id":3, "b":true}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/aa2caa8a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
index 7594783..82d51f1 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillAccessorList.java
@@ -23,6 +23,8 @@ import net.hydromatic.avatica.Cursor.Accessor;
 
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.BoundCheckingAccessor;
 import org.apache.drill.exec.vector.accessor.SqlAccessor;
 
 public class DrillAccessorList extends BasicList<Accessor>{
@@ -35,7 +37,8 @@ public class DrillAccessorList extends BasicList<Accessor>{
     int cnt = currentBatch.getSchema().getFieldCount();
     accessors = new Accessor[cnt];
     for(int i =0; i < cnt; i++){
-      SqlAccessor acc = TypeHelper.getSqlAccessor(currentBatch.getValueAccessorById(null, i).getValueVector());
+      final ValueVector vector = currentBatch.getValueAccessorById(null, i).getValueVector();
+      final SqlAccessor acc = new BoundCheckingAccessor(vector, TypeHelper.getSqlAccessor(vector));
       accessors[i] = new AvaticaDrillSqlAccessor(acc, cursor);
     }
   }