You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/07/14 17:43:18 UTC

[GitHub] sohami closed pull request #1371: DRILL-6588: Make Sys tables of nullable datatypes

sohami closed pull request #1371: DRILL-6588: Make Sys tables of nullable datatypes
URL: https://github.com/apache/drill/pull/1371
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 017e257f20e..1f5b93838f3 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 @@
    * @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 00000000000..18fca7cf83e
--- /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 7d8a8e49202..e91287925b1 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.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 @@
 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 PojoDataType(Class<?> pojoClass) {
       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 PojoDataType(Class<?> pojoClass) {
   }
 
   @Override
-  public List<SqlTypeName> getFieldSqlTypeNames() {
+  public List<SimpleImmutableEntry<SqlTypeName, Boolean>> getFieldSqlTypeNames() {
     return types;
   }
 
@@ -83,5 +86,4 @@ public PojoDataType(Class<?> pojoClass) {
   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 cffb486fb8b..146eac1a290 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.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.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 void remove() {
 
   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 a50a1c04542..1aab6579f33 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 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 DrillbitIterator(ExecutorFragmentContext c) {
   }
 
   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 a6c3b371b54..0df81210115 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.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 ExtendedOptionValueWrapper next() {
    */
   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 f39877d1e60..311e1c5c8dd 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 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 void remove() {
   }
 
   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 8760d1b5d6f..119426b2014 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.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 OptionValueWrapper next() {
    */
   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 09cc715fbf3..f748be32bc4 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 void remove() {
 
     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 9d9f236ace4..4ac2b122dd2 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.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 void remove() {
 
     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 a1845a259d3..0be65149ed2 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 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 void remove() {
   }
 
   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 4dd09f7f00d..b5c84c4466a 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 void testProfilesLimitPushDown() throws Exception {
     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));
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services