You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by so...@apache.org on 2018/07/14 17:43:20 UTC

[drill] branch master updated: DRILL-6588: Make Sys tables of nullable datatypes

This is an automated email from the ASF dual-hosted git repository.

sorabh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new bd6f63d  DRILL-6588: Make Sys tables of nullable datatypes
bd6f63d is described below

commit bd6f63ddf11a668d98e52444ba79c22494820996
Author: Kunal Khatua <kk...@users.noreply.github.com>
AuthorDate: Sat Jul 14 10:43:17 2018 -0700

    DRILL-6588: Make Sys tables of nullable datatypes
    
    This is to address the generic problem of NULL values being projected as a string because of all datatypes being non-nullable.
    This patch only applies to tables backed by the PojoReader (in our case, System tables). Added NonNullable annotations wherever application in any of the System tables, along with a unit test that verifies both nullable and non-nullable datatypes exist in the system tables.
    
    closes #1371
---
 .../apache/drill/exec/store/RecordDataType.java    | 25 ++++++++++-------
 .../apache/drill/exec/store/pojo/NonNullable.java  | 32 ++++++++++++++++++++++
 .../apache/drill/exec/store/pojo/PojoDataType.java | 30 ++++++++++----------
 .../store/sys/BitToUserConnectionIterator.java     |  7 ++++-
 .../drill/exec/store/sys/DrillbitIterator.java     |  5 ++++
 .../exec/store/sys/ExtendedOptionIterator.java     |  9 ++++--
 .../drill/exec/store/sys/MemoryIterator.java       |  2 ++
 .../drill/exec/store/sys/OptionIterator.java       | 13 +++++++--
 .../drill/exec/store/sys/ProfileInfoIterator.java  | 16 +++++++----
 .../drill/exec/store/sys/ProfileJsonIterator.java  |  2 ++
 .../drill/exec/store/sys/ThreadsIterator.java      |  2 ++
 .../drill/exec/store/sys/TestSystemTable.java      |  9 ++++++
 12 files changed, 116 insertions(+), 36 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordDataType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordDataType.java
index 017e257..1f5b938 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordDataType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordDataType.java
@@ -17,22 +17,23 @@
  */
 package org.apache.drill.exec.store;
 
-import com.google.common.collect.Lists;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlTypeName;
 
-import java.util.List;
-
 /**
  * RecordDataType defines names and data types of columns in a static drill table.
  */
 public abstract class RecordDataType {
 
   /**
-   * @return the {@link org.apache.calcite.sql.type.SqlTypeName} of columns in the table
+   * @return the {@link org.apache.calcite.sql.type.SqlTypeName} of columns in the table as a pair with its nullability
    */
-  public abstract List<SqlTypeName> getFieldSqlTypeNames();
+  public abstract List<SimpleImmutableEntry<SqlTypeName, Boolean>> getFieldSqlTypeNames();
 
   /**
    * @return the column names in the table
@@ -47,17 +48,21 @@ public abstract class RecordDataType {
    * @return the constructed type
    */
   public final RelDataType getRowType(RelDataTypeFactory factory) {
-    final List<SqlTypeName> types = getFieldSqlTypeNames();
+    final List<SimpleImmutableEntry<SqlTypeName, Boolean>> types = getFieldSqlTypeNames();
     final List<String> names = getFieldNames();
-    final List<RelDataType> fields = Lists.newArrayList();
-    for (final SqlTypeName typeName : types) {
+    final List<RelDataType> fields = new ArrayList<>();
+    for (SimpleImmutableEntry<SqlTypeName, Boolean> sqlTypePair : types) {
+      final SqlTypeName typeName = sqlTypePair.getKey();
+      final RelDataType tempDataType;
       switch (typeName) {
         case VARCHAR:
-          fields.add(factory.createSqlType(typeName, Integer.MAX_VALUE));
+          tempDataType = factory.createSqlType(typeName, Integer.MAX_VALUE);
           break;
         default:
-          fields.add(factory.createSqlType(typeName));
+          tempDataType = factory.createSqlType(typeName);
       }
+      //Add [Non]Nullable RelDataType
+      fields.add(factory.createTypeWithNullability(tempDataType, sqlTypePair.getValue()));
     }
     return factory.createStructType(fields, names);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/NonNullable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/NonNullable.java
new file mode 100644
index 0000000..18fca7c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/NonNullable.java
@@ -0,0 +1,32 @@
+/*
+ * 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.store.pojo;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Indicates NonNullable nature
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.FIELD)
+public @interface NonNullable {
+  boolean value() default true;
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
index 7d8a8e4..e912879 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
@@ -21,11 +21,11 @@ import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.calcite.sql.type.SqlTypeName;
-
-import com.google.common.collect.Lists;
 import org.apache.drill.exec.store.RecordDataType;
 
 /**
@@ -34,8 +34,8 @@ import org.apache.drill.exec.store.RecordDataType;
 public class PojoDataType extends RecordDataType {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoDataType.class);
 
-  private final List<SqlTypeName> types = Lists.newArrayList();
-  private final List<String> names = Lists.newArrayList();
+  private final List<SimpleImmutableEntry<SqlTypeName, Boolean>> types = new ArrayList<>();
+  private final List<String> names = new ArrayList<>();
   private final Class<?> pojoClass;
 
   public PojoDataType(Class<?> pojoClass) {
@@ -48,22 +48,25 @@ public class PojoDataType extends RecordDataType {
       Class<?> type = f.getType();
       names.add(f.getName());
 
+      //Absence of annotation @NonNullable => (isNullable=true)
+      final boolean isNullable = !(f.isAnnotationPresent(NonNullable.class));
+
       if (type == int.class || type == Integer.class) {
-        types.add(SqlTypeName.INTEGER);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.INTEGER, isNullable));
       } else if(type == boolean.class || type == Boolean.class) {
-        types.add(SqlTypeName.BOOLEAN);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.BOOLEAN, isNullable));
       } else if(type == long.class || type == Long.class) {
-        types.add(SqlTypeName.BIGINT);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.BIGINT, isNullable));
       } else if(type == double.class || type == Double.class) {
-        types.add(SqlTypeName.DOUBLE);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.DOUBLE, isNullable));
       } else if(type == BigDecimal.class) {
-        types.add(SqlTypeName.DECIMAL);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.DECIMAL, isNullable));
       } else if(type == String.class) {
-        types.add(SqlTypeName.VARCHAR);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.VARCHAR, isNullable));
       } else if(type.isEnum()) {
-        types.add(SqlTypeName.VARCHAR);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.VARCHAR, isNullable));
       } else if (type == Timestamp.class) {
-        types.add(SqlTypeName.TIMESTAMP);
+        types.add(new SimpleImmutableEntry<SqlTypeName, Boolean>(SqlTypeName.TIMESTAMP, isNullable));
       } else {
         throw new RuntimeException(String.format("PojoDataType doesn't yet support conversions from type [%s].", type));
       }
@@ -75,7 +78,7 @@ public class PojoDataType extends RecordDataType {
   }
 
   @Override
-  public List<SqlTypeName> getFieldSqlTypeNames() {
+  public List<SimpleImmutableEntry<SqlTypeName, Boolean>> getFieldSqlTypeNames() {
     return types;
   }
 
@@ -83,5 +86,4 @@ public class PojoDataType extends RecordDataType {
   public List<String> getFieldNames() {
     return names;
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/BitToUserConnectionIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/BitToUserConnectionIterator.java
index cffb486..146eac1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/BitToUserConnectionIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/BitToUserConnectionIterator.java
@@ -21,8 +21,8 @@ import java.sql.Timestamp;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.TimeZone;
 
 import org.apache.drill.exec.ExecConstants;
@@ -32,6 +32,7 @@ import org.apache.drill.exec.rpc.user.UserServer.BitToUserConnectionConfig;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.rest.profile.SimpleDurationFormat;
+import org.apache.drill.exec.store.pojo.NonNullable;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.joda.time.DateTime;
 
@@ -99,14 +100,18 @@ public class BitToUserConnectionIterator implements Iterator<Object> {
 
   public static class ConnectionInfo {
     public String user;
+    @NonNullable
     public String client;
+    @NonNullable
     public String drillbit;
+    @NonNullable
     public Timestamp established;
     public String duration;
     public int queries;
     public boolean isAuthenticated;
     public boolean isEncrypted;
     public boolean usingSSL;
+    @NonNullable
     public String session;
 
     public ConnectionInfo(Entry<BitToUserConnection, BitToUserConnectionConfig> connectionConfigPair, String hostname) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/DrillbitIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/DrillbitIterator.java
index a50a1c0..1aab657 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/DrillbitIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/DrillbitIterator.java
@@ -21,6 +21,7 @@ import java.util.Iterator;
 
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.pojo.NonNullable;
 
 public class DrillbitIterator implements Iterator<Object> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitIterator.class);
@@ -34,13 +35,17 @@ public class DrillbitIterator implements Iterator<Object> {
   }
 
   public static class DrillbitInstance {
+    @NonNullable
     public String hostname;
     public int user_port;
     public int control_port;
     public int data_port;
     public int http_port;
+    @NonNullable
     public boolean current;
+    @NonNullable
     public String version;
+    @NonNullable
     public String state;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
index a6c3b37..0df8121 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
@@ -24,13 +24,14 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-
-import com.google.common.collect.Lists;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.OptionValue.Kind;
 import org.apache.drill.exec.server.options.OptionValue.OptionScope;
+import org.apache.drill.exec.store.pojo.NonNullable;
+
+import com.google.common.collect.Lists;
 
 /*
  * Extends the original Option iterator. The idea is to hide the implementation details and present the
@@ -134,10 +135,14 @@ public class ExtendedOptionIterator implements Iterator<Object> {
    */
   public static class ExtendedOptionValueWrapper {
 
+    @NonNullable
     public final String name;
+    @NonNullable
     public final String kind;
+    @NonNullable
     public final OptionValue.AccessibleScopes accessibleScopes;
     public final String val;
+    @NonNullable
     public final OptionScope optionScope;
 
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
index f39877d..311e1c5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
@@ -27,6 +27,7 @@ import java.util.List;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.pojo.NonNullable;
 
 public class MemoryIterator implements Iterator<Object> {
 
@@ -83,6 +84,7 @@ public class MemoryIterator implements Iterator<Object> {
   }
 
   public static class MemoryInfo {
+    @NonNullable
     public String hostname;
     public long user_port;
     public long heap_current;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/OptionIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/OptionIterator.java
index 8760d1b..119426b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/OptionIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/OptionIterator.java
@@ -21,15 +21,17 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.server.options.DrillConfigIterator;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.OptionValue.AccessibleScopes;
 import org.apache.drill.exec.server.options.OptionValue.Kind;
 import org.apache.drill.exec.server.options.OptionValue.OptionScope;
-import org.apache.drill.exec.server.options.OptionValue.AccessibleScopes;
+import org.apache.drill.exec.store.pojo.NonNullable;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
 
 public class OptionIterator implements Iterator<Object> {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OptionIterator.class);
@@ -104,10 +106,15 @@ public class OptionIterator implements Iterator<Object> {
    */
   public static class OptionValueWrapper {
 
+    @NonNullable
     public final String name;
+    @NonNullable
     public final Kind kind;
+    @NonNullable
     public final AccessibleScopes accessibleScopes;
+    @NonNullable
     public final OptionScope optionScope;
+    @NonNullable
     public final Status status;
     public final Long num_val;
     public final String string_val;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileInfoIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileInfoIterator.java
index 09cc715..f748be3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileInfoIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileInfoIterator.java
@@ -17,17 +17,19 @@
  */
 package org.apache.drill.exec.store.sys;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Iterators;
+import java.sql.Timestamp;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import javax.annotation.Nullable;
 
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.store.pojo.NonNullable;
 
-import javax.annotation.Nullable;
-import java.sql.Timestamp;
-import java.util.Iterator;
-import java.util.Map.Entry;
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
 
 /**
  * System table listing completed profiles
@@ -114,8 +116,10 @@ public class ProfileInfoIterator extends ProfileIterator {
 
     private static final ProfileInfo DEFAULT = new ProfileInfo();
 
+    @NonNullable
     public final String queryId;
     public final Timestamp startTime;
+    @NonNullable
     public final String foreman;
     public final long fragments;
     public final String user;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileJsonIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileJsonIterator.java
index 9d9f236..4ac2b12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileJsonIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ProfileJsonIterator.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
 import org.apache.drill.exec.serialization.InstanceSerializer;
+import org.apache.drill.exec.store.pojo.NonNullable;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterators;
@@ -114,6 +115,7 @@ public class ProfileJsonIterator extends ProfileIterator {
 
     private static final ProfileJson DEFAULT = new ProfileJson();
 
+    @NonNullable
     public final String queryId;
     public final String json;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ThreadsIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ThreadsIterator.java
index a1845a2..0be6514 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ThreadsIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ThreadsIterator.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.pojo.NonNullable;
 
 public class ThreadsIterator implements Iterator<Object> {
 
@@ -62,6 +63,7 @@ public class ThreadsIterator implements Iterator<Object> {
   }
 
   public static class ThreadsInfo {
+    @NonNullable
     public String hostname;
     public long user_port;
     public long total_threads;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
index 4dd09f7..b5c84c4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.store.sys;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.exec.ExecConstants;
@@ -90,4 +92,11 @@ public class TestSystemTable extends PlanTestBase {
     String numFilesPattern = "maxRecordsToRead=10";
     testPlanMatchingPatterns(query, new String[] {numFilesPattern}, new String[] {});
   }
+
+  @Test
+  public void testColumnNullability() throws Exception {
+    String query = "select distinct is_nullable, count(*) from INFORMATION_SCHEMA.`COLUMNS` where table_schema = 'sys' group by is_nullable";
+    //Asserting a mix of nullable and non-nullable columns (pre-DRILL-6588, all columns were Not Nullable)
+    assertEquals(2, testSql(query));
+  }
 }