[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-03-02 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-21 Thread laurentgo
Github user laurentgo commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r102364337
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java ---
@@ -78,19 +111,237 @@ private MetaResultSet s(String s) {
 }
   }
 
+  /** Information about type mapping. */
+  private static class TypeInfo {
+private static Map MAPPING = 
ImmutableMap. builder()
+.put(boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Byte.TYPE, of(Types.TINYINT, "TINYINT"))
+.put(Byte.class, of(Types.TINYINT, "TINYINT"))
+.put(Short.TYPE, of(Types.SMALLINT, "SMALLINT"))
+.put(Short.class, of(Types.SMALLINT, "SMALLINT"))
+.put(Integer.TYPE, of(Types.INTEGER, "INTEGER"))
+.put(Integer.class, of(Types.INTEGER, "INTEGER"))
+.put(Long.TYPE,  of(Types.BIGINT, "BIGINT"))
+.put(Long.class, of(Types.BIGINT, "BIGINT"))
+.put(Float.TYPE, of(Types.FLOAT, "FLOAT"))
+.put(Float.class,  of(Types.FLOAT, "FLOAT"))
+.put(Double.TYPE,  of(Types.DOUBLE, "DOUBLE"))
+.put(Double.class, of(Types.DOUBLE, "DOUBLE"))
+.put(String.class, of(Types.VARCHAR, "CHARACTER VARYING"))
+.put(java.sql.Date.class, of(Types.DATE, "DATE"))
+.put(Time.class, of(Types.TIME, "TIME"))
+.put(Timestamp.class, of(Types.TIMESTAMP, "TIMESTAMP"))
+.build();
+
+private final int sqlType;
+private final String sqlTypeName;
+
+public TypeInfo(int sqlType, String sqlTypeName) {
+  this.sqlType = sqlType;
+  this.sqlTypeName = sqlTypeName;
+}
 
+private static TypeInfo of(int sqlType, String sqlTypeName) {
+  return new TypeInfo(sqlType, sqlTypeName);
+}
 
-  @Override
-  protected  MetaResultSet createEmptyResultSet(Class clazz) {
-return s(
-"SELECT '' AS `Interim zero-row result set` "  // dummy row type
-+ "FROM INFORMATION_SCHEMA.CATALOGS "  // any table
-+ "LIMIT 0"// zero rows
-);
+public static TypeInfo get(Class clazz) {
+  return MAPPING.get(clazz);
+}
   }
 
-  @Override
-  public MetaResultSet getTables(String catalog, final Pat schemaPattern, 
final Pat tableNamePattern,
+  /** Metadata describing a column.
+   * Copied from Avatica with several fixes
+   * */
+  public static class MetaColumn implements Named {
+public final String tableCat;
+public final String tableSchem;
+public final String tableName;
+public final String columnName;
+public final int dataType;
+public final String typeName;
+public final Integer columnSize;
+public final Integer bufferLength = null;
+public final Integer decimalDigits;
+public final Integer numPrecRadix;
+public final int nullable;
+public final String remarks = null;
+public final String columnDef = null;
+public final Integer sqlDataType = null;
+public final Integer sqlDatetimeSub = null;
+public final Integer charOctetLength;
+public final int ordinalPosition;
+@NotNull
+public final String isNullable;
+public final String scopeCatalog = null;
+public final String scopeSchema = null;
+public final String scopeTable = null;
+public final Short sourceDataType = null;
+@NotNull
+public final String isAutoincrement = "";
+@NotNull
+public final String isGeneratedcolumn = "";
+
+public MetaColumn(
+String tableCat,
+String tableSchem,
+String tableName,
+String columnName,
+int dataType,
+String typeName,
+Integer columnSize,
+Integer decimalDigits,
+Integer numPrecRadix,
+int nullable,
+Integer charOctetLength,
+int ordinalPosition,
+String isNullable) {
+  this.tableCat = tableCat;
+  this.tableSchem = tableSchem;
+  this.tableName = tableName;
+  this.columnName = columnName;
+  this.dataType = dataType;
+  this.typeName = typeName;
+  this.columnSize = columnSize;
+  this.decimalDigits = decimalDigits;
+  this.numPrecRadix = numPrecRadix;
+  this.nullable = nullable;
+  this.charOctetLength = charOctetLength;
+  this.ordinalPosition = ordinalPosition;
+  this.isNullable = isNullable;
+}
+
+@Override
+public String getName() {
+  return columnName;
+}
  

[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-21 Thread laurentgo
Github user laurentgo commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r102363803
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
@@ -24,33 +24,260 @@
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.Meta.Signature;
 import org.apache.calcite.avatica.util.ArrayImpl.Factory;
 import org.apache.calcite.avatica.util.Cursor;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
+import org.apache.drill.jdbc.SchemaChangeListener;
 import org.slf4j.Logger;
 
+import com.google.common.collect.Queues;
+
 
 class DrillCursor implements Cursor {
+
+  
+  // ResultsListener:
+  static class ResultsListener implements UserResultsListener {
+private static final org.slf4j.Logger logger =
+org.slf4j.LoggerFactory.getLogger(ResultsListener.class);
+
+private static volatile int nextInstanceId = 1;
+
+/** (Just for logging.) */
+private final int instanceId;
+
+private final int batchQueueThrottlingThreshold;
+
+/** (Just for logging.) */
+private volatile QueryId queryId;
+
+/** (Just for logging.) */
+private int lastReceivedBatchNumber;
+/** (Just for logging.) */
+private int lastDequeuedBatchNumber;
+
+private volatile UserException executionFailureException;
+
+// TODO:  Revisit "completed".  Determine and document exactly what it
+// means.  Some uses imply that it means that incoming messages 
indicate
+// that the _query_ has _terminated_ (not necessarily _completing_
+// normally), while some uses imply that it's some other state of the
+// ResultListener.  Some uses seem redundant.)
+volatile boolean completed = false;
+
+/** Whether throttling of incoming data is active. */
+private final AtomicBoolean throttled = new AtomicBoolean( false );
+private volatile ConnectionThrottle throttle;
+
+private volatile boolean closed = false;
+
+private final CountDownLatch firstMessageReceived = new 
CountDownLatch(1);
+
+final LinkedBlockingDeque batchQueue =
+Queues.newLinkedBlockingDeque();
+
+
+/**
+ * ...
+ * @param  batchQueueThrottlingThreshold
+ * queue size threshold for throttling server
+ */
+ResultsListener( int batchQueueThrottlingThreshold ) {
+  instanceId = nextInstanceId++;
+  this.batchQueueThrottlingThreshold = batchQueueThrottlingThreshold;
+  logger.debug( "[#{}] Query listener created.", instanceId );
+}
+
+/**
+ * Starts throttling if not currently throttling.
+ * @param  throttle  the "throttlable" object to throttle
+ * @return  true if actually started (wasn't throttling already)
+ */
+private boolean startThrottlingIfNot( ConnectionThrottle throttle ) {
+  final boolean started = throttled.compareAndSet( false, true );
+  if ( started ) {
+this.throttle = throttle;
+throttle.setAutoRead(false);
+  }
+  return started;
+}
+
+/**
+ * Stops throttling if currently throttling.
+ * @return  true if actually stopped (was throttling)
+ */
+private boolean stopThrottlingIfSo() {
+  final boolean stopped = throttled.compareAndSet( true, false );
+  if ( stopped ) {
+throttle.setAutoRead(true);

[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-21 Thread laurentgo
Github user laurentgo commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r102363018
  
--- Diff: 
exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
 ---
@@ -2712,22 +2712,18 @@ public void 
test_SOURCE_DATA_TYPE_hasSameNameAndLabel() throws SQLException {
 
   @Test
   public void test_SOURCE_DATA_TYPE_hasRightTypeString() throws 
SQLException {
-// TODO(DRILL-2135):  Resolve workaround:
-//assertThat( rsMetadata.getColumnTypeName( 22 ), equalTo( "SMALLINT" 
) );
-assertThat( rowsMetadata.getColumnTypeName( 22 ), equalTo( "INTEGER" ) 
);
+assertThat( rowsMetadata.getColumnTypeName( 22 ), equalTo( "SMALLINT" 
) );
--- End diff --

no, it isn't unfortunately


---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r100428330
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java ---
@@ -78,19 +111,237 @@ private MetaResultSet s(String s) {
 }
   }
 
+  /** Information about type mapping. */
+  private static class TypeInfo {
+private static Map MAPPING = 
ImmutableMap. builder()
+.put(boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Byte.TYPE, of(Types.TINYINT, "TINYINT"))
+.put(Byte.class, of(Types.TINYINT, "TINYINT"))
+.put(Short.TYPE, of(Types.SMALLINT, "SMALLINT"))
+.put(Short.class, of(Types.SMALLINT, "SMALLINT"))
+.put(Integer.TYPE, of(Types.INTEGER, "INTEGER"))
+.put(Integer.class, of(Types.INTEGER, "INTEGER"))
+.put(Long.TYPE,  of(Types.BIGINT, "BIGINT"))
+.put(Long.class, of(Types.BIGINT, "BIGINT"))
+.put(Float.TYPE, of(Types.FLOAT, "FLOAT"))
+.put(Float.class,  of(Types.FLOAT, "FLOAT"))
+.put(Double.TYPE,  of(Types.DOUBLE, "DOUBLE"))
+.put(Double.class, of(Types.DOUBLE, "DOUBLE"))
+.put(String.class, of(Types.VARCHAR, "CHARACTER VARYING"))
+.put(java.sql.Date.class, of(Types.DATE, "DATE"))
+.put(Time.class, of(Types.TIME, "TIME"))
+.put(Timestamp.class, of(Types.TIMESTAMP, "TIMESTAMP"))
+.build();
+
+private final int sqlType;
+private final String sqlTypeName;
+
+public TypeInfo(int sqlType, String sqlTypeName) {
+  this.sqlType = sqlType;
+  this.sqlTypeName = sqlTypeName;
+}
 
+private static TypeInfo of(int sqlType, String sqlTypeName) {
+  return new TypeInfo(sqlType, sqlTypeName);
+}
 
-  @Override
-  protected  MetaResultSet createEmptyResultSet(Class clazz) {
-return s(
-"SELECT '' AS `Interim zero-row result set` "  // dummy row type
-+ "FROM INFORMATION_SCHEMA.CATALOGS "  // any table
-+ "LIMIT 0"// zero rows
-);
+public static TypeInfo get(Class clazz) {
+  return MAPPING.get(clazz);
+}
   }
 
-  @Override
-  public MetaResultSet getTables(String catalog, final Pat schemaPattern, 
final Pat tableNamePattern,
+  /** Metadata describing a column.
+   * Copied from Avatica with several fixes
+   * */
+  public static class MetaColumn implements Named {
+public final String tableCat;
+public final String tableSchem;
+public final String tableName;
+public final String columnName;
+public final int dataType;
+public final String typeName;
+public final Integer columnSize;
+public final Integer bufferLength = null;
+public final Integer decimalDigits;
+public final Integer numPrecRadix;
+public final int nullable;
+public final String remarks = null;
+public final String columnDef = null;
+public final Integer sqlDataType = null;
+public final Integer sqlDatetimeSub = null;
+public final Integer charOctetLength;
+public final int ordinalPosition;
+@NotNull
+public final String isNullable;
+public final String scopeCatalog = null;
+public final String scopeSchema = null;
+public final String scopeTable = null;
+public final Short sourceDataType = null;
+@NotNull
+public final String isAutoincrement = "";
+@NotNull
+public final String isGeneratedcolumn = "";
+
+public MetaColumn(
+String tableCat,
+String tableSchem,
+String tableName,
+String columnName,
+int dataType,
+String typeName,
+Integer columnSize,
+Integer decimalDigits,
+Integer numPrecRadix,
+int nullable,
+Integer charOctetLength,
+int ordinalPosition,
+String isNullable) {
+  this.tableCat = tableCat;
+  this.tableSchem = tableSchem;
+  this.tableName = tableName;
+  this.columnName = columnName;
+  this.dataType = dataType;
+  this.typeName = typeName;
+  this.columnSize = columnSize;
+  this.decimalDigits = decimalDigits;
+  this.numPrecRadix = numPrecRadix;
+  this.nullable = nullable;
+  this.charOctetLength = charOctetLength;
+  this.ordinalPosition = ordinalPosition;
+  this.isNullable = isNullable;
+}
+
+@Override
+public String getName() {
+  return columnName;
+   

[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r86650548
  
--- Diff: 
exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
 ---
@@ -2712,22 +2712,18 @@ public void 
test_SOURCE_DATA_TYPE_hasSameNameAndLabel() throws SQLException {
 
   @Test
   public void test_SOURCE_DATA_TYPE_hasRightTypeString() throws 
SQLException {
-// TODO(DRILL-2135):  Resolve workaround:
-//assertThat( rsMetadata.getColumnTypeName( 22 ), equalTo( "SMALLINT" 
) );
-assertThat( rowsMetadata.getColumnTypeName( 22 ), equalTo( "INTEGER" ) 
);
+assertThat( rowsMetadata.getColumnTypeName( 22 ), equalTo( "SMALLINT" 
) );
--- End diff --

Is [DRILL-2135](https://issues.apache.org/jira/browse/DRILL-2135) being 
resolved as well?



---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r86275226
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java ---
@@ -61,77 +91,271 @@ private static Signature newSignature(String sql) {
 sql,
 Collections. emptyList(),
 Collections.emptyMap(),
-CursorFactory.OBJECT);
+null // CursorFactory set to null, as SQL requests use DrillCursor
+);
   }
 
-  private MetaResultSet s(String s) {
-try {
-  logger.debug("Running {}", s);
-
-  AvaticaStatement statement = connection.createStatement();
-  return MetaResultSet.create(connection.id, statement.getId(), true,
-  newSignature(s), null);
-} catch (Exception e) {
-  // Wrap in RuntimeException because Avatica's abstract method 
declarations
-  // didn't allow for SQLException!
-  throw new DrillRuntimeException("Failure while attempting to get 
DatabaseMetadata.", e);
+
+  /** Information about type mapping. */
+  private static class TypeInfo {
+private static Map MAPPING = 
ImmutableMap. builder()
+.put(boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
--- End diff --

Given this, how about `int.class` and others?



---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r86651605
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ops/ViewExpansionContext.java
 ---
@@ -70,20 +73,25 @@
 public class ViewExpansionContext {
   private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ViewExpansionContext.class);
 
-  private final QueryContext queryContext;
+  private SchemaConfigInfoProvider schemaConfigInfoProvider;
--- End diff --

final



---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r86651825
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
@@ -24,33 +24,260 @@
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.Meta.Signature;
 import org.apache.calcite.avatica.util.ArrayImpl.Factory;
 import org.apache.calcite.avatica.util.Cursor;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.store.ischema.InfoSchemaConstants;
+import org.apache.drill.jdbc.SchemaChangeListener;
 import org.slf4j.Logger;
 
+import com.google.common.collect.Queues;
+
 
 class DrillCursor implements Cursor {
+
+  
+  // ResultsListener:
+  static class ResultsListener implements UserResultsListener {
+private static final org.slf4j.Logger logger =
+org.slf4j.LoggerFactory.getLogger(ResultsListener.class);
+
+private static volatile int nextInstanceId = 1;
+
+/** (Just for logging.) */
+private final int instanceId;
+
+private final int batchQueueThrottlingThreshold;
+
+/** (Just for logging.) */
+private volatile QueryId queryId;
+
+/** (Just for logging.) */
+private int lastReceivedBatchNumber;
+/** (Just for logging.) */
+private int lastDequeuedBatchNumber;
+
+private volatile UserException executionFailureException;
+
+// TODO:  Revisit "completed".  Determine and document exactly what it
+// means.  Some uses imply that it means that incoming messages 
indicate
+// that the _query_ has _terminated_ (not necessarily _completing_
+// normally), while some uses imply that it's some other state of the
+// ResultListener.  Some uses seem redundant.)
+volatile boolean completed = false;
+
+/** Whether throttling of incoming data is active. */
+private final AtomicBoolean throttled = new AtomicBoolean( false );
+private volatile ConnectionThrottle throttle;
+
+private volatile boolean closed = false;
+
+private final CountDownLatch firstMessageReceived = new 
CountDownLatch(1);
+
+final LinkedBlockingDeque batchQueue =
+Queues.newLinkedBlockingDeque();
+
+
+/**
+ * ...
+ * @param  batchQueueThrottlingThreshold
+ * queue size threshold for throttling server
+ */
+ResultsListener( int batchQueueThrottlingThreshold ) {
+  instanceId = nextInstanceId++;
+  this.batchQueueThrottlingThreshold = batchQueueThrottlingThreshold;
+  logger.debug( "[#{}] Query listener created.", instanceId );
+}
+
+/**
+ * Starts throttling if not currently throttling.
+ * @param  throttle  the "throttlable" object to throttle
+ * @return  true if actually started (wasn't throttling already)
+ */
+private boolean startThrottlingIfNot( ConnectionThrottle throttle ) {
+  final boolean started = throttled.compareAndSet( false, true );
+  if ( started ) {
+this.throttle = throttle;
+throttle.setAutoRead(false);
+  }
+  return started;
+}
+
+/**
+ * Stops throttling if currently throttling.
+ * @return  true if actually stopped (was throttling)
+ */
+private boolean stopThrottlingIfSo() {
+  final boolean stopped = throttled.compareAndSet( true, false );
+  if ( stopped ) {
+throttle.setAutoRead(true);

[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r100425035
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java ---
@@ -78,19 +111,237 @@ private MetaResultSet s(String s) {
 }
   }
 
+  /** Information about type mapping. */
+  private static class TypeInfo {
+private static Map MAPPING = 
ImmutableMap. builder()
+.put(boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Boolean.class, of(Types.BOOLEAN, "BOOLEAN"))
+.put(Byte.TYPE, of(Types.TINYINT, "TINYINT"))
+.put(Byte.class, of(Types.TINYINT, "TINYINT"))
+.put(Short.TYPE, of(Types.SMALLINT, "SMALLINT"))
+.put(Short.class, of(Types.SMALLINT, "SMALLINT"))
+.put(Integer.TYPE, of(Types.INTEGER, "INTEGER"))
+.put(Integer.class, of(Types.INTEGER, "INTEGER"))
+.put(Long.TYPE,  of(Types.BIGINT, "BIGINT"))
+.put(Long.class, of(Types.BIGINT, "BIGINT"))
+.put(Float.TYPE, of(Types.FLOAT, "FLOAT"))
+.put(Float.class,  of(Types.FLOAT, "FLOAT"))
+.put(Double.TYPE,  of(Types.DOUBLE, "DOUBLE"))
+.put(Double.class, of(Types.DOUBLE, "DOUBLE"))
+.put(String.class, of(Types.VARCHAR, "CHARACTER VARYING"))
+.put(java.sql.Date.class, of(Types.DATE, "DATE"))
+.put(Time.class, of(Types.TIME, "TIME"))
+.put(Timestamp.class, of(Types.TIMESTAMP, "TIMESTAMP"))
+.build();
+
+private final int sqlType;
+private final String sqlTypeName;
+
+public TypeInfo(int sqlType, String sqlTypeName) {
+  this.sqlType = sqlType;
+  this.sqlTypeName = sqlTypeName;
+}
 
+private static TypeInfo of(int sqlType, String sqlTypeName) {
+  return new TypeInfo(sqlType, sqlTypeName);
+}
 
-  @Override
-  protected  MetaResultSet createEmptyResultSet(Class clazz) {
-return s(
-"SELECT '' AS `Interim zero-row result set` "  // dummy row type
-+ "FROM INFORMATION_SCHEMA.CATALOGS "  // any table
-+ "LIMIT 0"// zero rows
-);
+public static TypeInfo get(Class clazz) {
+  return MAPPING.get(clazz);
+}
   }
 
-  @Override
-  public MetaResultSet getTables(String catalog, final Pat schemaPattern, 
final Pat tableNamePattern,
+  /** Metadata describing a column.
+   * Copied from Avatica with several fixes
+   * */
+  public static class MetaColumn implements Named {
+public final String tableCat;
+public final String tableSchem;
+public final String tableName;
+public final String columnName;
+public final int dataType;
+public final String typeName;
+public final Integer columnSize;
+public final Integer bufferLength = null;
+public final Integer decimalDigits;
+public final Integer numPrecRadix;
+public final int nullable;
+public final String remarks = null;
+public final String columnDef = null;
+public final Integer sqlDataType = null;
+public final Integer sqlDatetimeSub = null;
+public final Integer charOctetLength;
+public final int ordinalPosition;
+@NotNull
+public final String isNullable;
+public final String scopeCatalog = null;
+public final String scopeSchema = null;
+public final String scopeTable = null;
+public final Short sourceDataType = null;
+@NotNull
+public final String isAutoincrement = "";
+@NotNull
+public final String isGeneratedcolumn = "";
+
+public MetaColumn(
+String tableCat,
+String tableSchem,
+String tableName,
+String columnName,
+int dataType,
+String typeName,
+Integer columnSize,
+Integer decimalDigits,
+Integer numPrecRadix,
+int nullable,
+Integer charOctetLength,
+int ordinalPosition,
+String isNullable) {
+  this.tableCat = tableCat;
+  this.tableSchem = tableSchem;
+  this.tableName = tableName;
+  this.columnName = columnName;
+  this.dataType = dataType;
+  this.typeName = typeName;
+  this.columnSize = columnSize;
+  this.decimalDigits = decimalDigits;
+  this.numPrecRadix = numPrecRadix;
+  this.nullable = nullable;
+  this.charOctetLength = charOctetLength;
+  this.ordinalPosition = ordinalPosition;
+  this.isNullable = isNullable;
+}
+
+@Override
+public String getName() {
+  return columnName;
+   

[GitHub] drill pull request #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2017-02-14 Thread sudheeshkatkam
Github user sudheeshkatkam commented on a diff in the pull request:

https://github.com/apache/drill/pull/613#discussion_r100422973
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java ---
@@ -78,19 +111,237 @@ private MetaResultSet s(String s) {
 }
   }
 
+  /** Information about type mapping. */
+  private static class TypeInfo {
+private static Map MAPPING = 
ImmutableMap. builder()
--- End diff --

final


---
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 #613: DRILL-4730: Update JDBC DatabaseMetaData implementa...

2016-10-12 Thread laurentgo
GitHub user laurentgo opened a pull request:

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

DRILL-4730: Update JDBC DatabaseMetaData implementation to use new Metadata 
APIs

Update JDBC driver to use Metadata APIs instead of executing SQL queries

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

$ git pull https://github.com/laurentgo/drill laurent/DRILL-4730

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

https://github.com/apache/drill/pull/613.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 #613






---
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.
---