http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
new file mode 100644
index 0000000..5119212
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
@@ -0,0 +1,81 @@
+/**
+ * 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.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+
+import net.hydromatic.avatica.ColumnMetaData;
+
+public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillColumnMetaDataList.class);
+
+ private ColumnMetaData[] columns = new ColumnMetaData[0];
+
+ @Override
+ public int size() {
+ return columns.length;
+ }
+
+ @Override
+ public ColumnMetaData get(int index) {
+ return columns[index];
+ }
+
+ public void updateColumnMetaData(String catalogName, String schemaName, String tableName, BatchSchema schema){
+
+ columns = new ColumnMetaData[schema.getFieldCount()];
+ for(int i = 0; i < schema.getFieldCount(); i++){
+ MaterializedField f = schema.getColumn(i);
+ MajorType t = f.getType();
+ ColumnMetaData col = new ColumnMetaData( //
+ i, // ordinal
+ false, // autoIncrement
+ true, // caseSensitive
+ false, // searchable
+ false, // currency
+ f.getDataMode() == DataMode.OPTIONAL ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls, //nullability
+ !Types.isUnSigned(t), // signed
+ 10, // display size.
+ f.getName(), // label
+ f.getName(), // columnname
+ schemaName, // schemaname
+ t.hasPrecision() ? t.getPrecision() : 0, // precision
+ t.hasScale() ? t.getScale() : 0, // scale
+ null, // tablename is null so sqlline doesn't try to retrieve primary keys.
+ catalogName, // catalogname
+ Types.getSqlType(t), // sql type
+ t.getMinorType().name()+":" + t.getMode().name(), // typename
+ true, // readonly
+ false, // writable
+ false, // definitely writable
+ "none", // column class name
+ ColumnMetaData.Rep.BOOLEAN // Dummy value for representation as it doesn't apply in drill's case.
+ );
+ columns[i] =col;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
new file mode 100644
index 0000000..ab1259a
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
@@ -0,0 +1,41 @@
+/**
+ * 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.jdbc;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.apache.drill.exec.client.DrillClient;
+
+
+public interface DrillConnection extends Connection{
+
+ // in java.sql.Connection from JDK 1.7, but declare here to allow other JDKs
+ void setSchema(String schema) throws SQLException;
+
+ // in java.sql.Connection from JDK 1.7, but declare here to allow other JDKs
+ String getSchema() throws SQLException;
+
+ /** Returns a view onto this connection's configuration properties. Code
+ * within Optiq should use this view rather than calling
+ * {@link java.util.Properties#getProperty(String)}. */
+ ConnectionConfig config();
+
+ public DrillClient getClient();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
new file mode 100644
index 0000000..11a1157
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -0,0 +1,166 @@
+/**
+ * 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.jdbc;
+
+import java.io.IOException;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaFactory;
+import net.hydromatic.avatica.Helper;
+import net.hydromatic.avatica.Meta;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+
+/**
+ * Implementation of JDBC connection in Drill.
+ *
+ * <p>
+ * Abstract to allow newer versions of JDBC to add methods.
+ * </p>
+ */
+abstract class DrillConnectionImpl extends AvaticaConnection implements org.apache.drill.jdbc.DrillConnection {
+ public final DrillStatementRegistry registry = new DrillStatementRegistry();
+ final ConnectionConfig config;
+
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConnection.class);
+
+ private final DrillClient client;
+ private final BufferAllocator allocator;
+ private Drillbit bit;
+ private RemoteServiceSet serviceSet;
+
+ protected DrillConnectionImpl(Driver driver, AvaticaFactory factory, String url, Properties info) throws SQLException{
+ super(driver, factory, url, info);
+ this.config = new ConnectionConfig(info);
+
+ this.allocator = new TopLevelAllocator();
+
+ try{
+ if(config.isLocal()){
+ RemoteServiceSet set = GlobalServiceSetReference.SETS.get();
+ if(set == null){
+ // we're embedded, start a local drill bit.
+ serviceSet = RemoteServiceSet.getLocalServiceSet();
+ set = serviceSet;
+ try{
+ bit = new Drillbit(driver.getConfig(), serviceSet);
+ bit.run();
+ }catch(Exception e){
+ throw new SQLException("Failure while attempting to start Drillbit in embedded mode.", e);
+ }
+ }else{
+ serviceSet = null;
+ bit = null;
+ }
+ this.client = new DrillClient(driver.getConfig(), set.getCoordinator());
+ this.client.connect(null);
+ }else{
+ this.client = new DrillClient();
+ this.client.connect(config.getZookeeperConnectionString());
+ }
+ }catch(RpcException e){
+ throw new SQLException("Failure while attempting to connect to Drill.", e);
+ }
+ }
+
+
+ public ConnectionConfig config(){
+ return config;
+ }
+
+ @Override
+ protected Meta createMeta() {
+ return new MetaImpl(this);
+ }
+
+ MetaImpl meta() {
+ return (MetaImpl) meta;
+ }
+
+ BufferAllocator getAllocator(){
+ return allocator;
+ }
+
+ public DrillClient getClient(){
+ return client;
+ }
+
+ @Override
+ public DrillStatement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability)
+ throws SQLException {
+ DrillStatement statement = (DrillStatement) super.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
+ registry.addStatement(statement);
+ return statement;
+ }
+
+ @Override
+ public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency,
+ int resultSetHoldability) throws SQLException {
+ try {
+ DrillPrepareResult prepareResult = new DrillPrepareResult(sql);
+ DrillPreparedStatement statement = (DrillPreparedStatement) factory.newPreparedStatement(this, prepareResult,
+ resultSetType, resultSetConcurrency, resultSetHoldability);
+ registry.addStatement(statement);
+ return statement;
+ } catch (RuntimeException e) {
+ throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
+ } catch (Exception e) {
+ throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
+ }
+ }
+
+ @Override
+ public TimeZone getTimeZone() {
+ return config.getTimeZone();
+ }
+
+ // do not make public
+ UnregisteredDriver getDriver() {
+ return driver;
+ }
+
+ // do not make public
+ AvaticaFactory getFactory() {
+ return factory;
+ }
+
+ void cleanup(){
+ client.close();
+ allocator.close();
+ if(bit != null) bit.close();
+
+ if(serviceSet != null){
+ try{
+ serviceSet.close();
+ }catch(IOException e){
+ logger.warn("Exception while closing service set.", e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillCursor.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillCursor.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillCursor.java
new file mode 100644
index 0000000..1145b84
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillCursor.java
@@ -0,0 +1,127 @@
+/**
+ * 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.jdbc;
+
+import java.sql.SQLException;
+import java.util.Calendar;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.Cursor;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.jdbc.DrillResultSet.Listener;
+
+public class DrillCursor implements Cursor{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCursor.class);
+
+ private static final String UNKNOWN = "--UNKNOWN--";
+
+ private boolean started = false;
+ private boolean finished = false;
+ private final RecordBatchLoader currentBatch;
+ private final DrillResultSet.Listener listener;
+ private boolean redoFirstNext = false;
+
+ private DrillColumnMetaDataList columnMetaDataList;
+
+ final DrillResultSet results;
+ int currentRecord = 0;
+ private long recordBatchCount;
+ private final DrillAccessorList accessors = new DrillAccessorList();
+
+
+ public DrillCursor(DrillResultSet results) {
+ super();
+ this.results = results;
+ currentBatch = results.currentBatch;
+ this.listener = results.listener;
+ }
+
+ @Override
+ public List<Accessor> createAccessors(List<ColumnMetaData> types, Calendar localCalendar) {
+ columnMetaDataList = (DrillColumnMetaDataList) types;
+ return accessors;
+ }
+
+ @Override
+ public boolean next() throws SQLException {
+ if(!started){
+ started = true;
+ redoFirstNext = true;
+ }else if(redoFirstNext && !finished){
+ redoFirstNext = false;
+ return true;
+ }
+
+ if(finished) return false;
+
+ if(currentRecord+1 < currentBatch.getRecordCount()){
+ currentRecord++;
+ return true;
+ }else{
+ try {
+ QueryResultBatch qrb = listener.getNext();
+ recordBatchCount++;
+ while(qrb != null && qrb.getHeader().getRowCount() == 0 ){
+ qrb.release();
+ qrb = listener.getNext();
+ recordBatchCount++;
+ }
+
+ if(qrb == null){
+ finished = true;
+ return false;
+ }else{
+ currentRecord = 0;
+ boolean changed = currentBatch.load(qrb.getHeader().getDef(), qrb.getData());
+ if(changed) updateColumns();
+ return true;
+ }
+ } catch (RpcException | InterruptedException | SchemaChangeException e) {
+ throw new SQLException("Failure while trying to get next result batch.", e);
+ }
+
+ }
+ }
+
+ void updateColumns(){
+ accessors.generateAccessors(this, currentBatch);
+ columnMetaDataList.updateColumnMetaData(UNKNOWN, UNKNOWN, UNKNOWN, currentBatch.getSchema());
+ if(results.changeListener != null) results.changeListener.schemaChanged(currentBatch.getSchema());
+ }
+
+ public long getRecordBatchCount(){
+ return recordBatchCount;
+ }
+
+ @Override
+ public void close() {
+ results.cleanup();
+ }
+
+ @Override
+ public boolean wasNull() throws SQLException {
+ return accessors.wasNull();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillDriverVersion.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillDriverVersion.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillDriverVersion.java
deleted file mode 100644
index e395515..0000000
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillDriverVersion.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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.jdbc;
-
-import net.hydromatic.optiq.jdbc.DriverVersion;
-
-/**
- * Version information for Drill JDBC Driver.
- */
-class DrillDriverVersion extends DriverVersion {
- /** Creates a DrillDriverVersion. */
- DrillDriverVersion() {
- super(
- "Apache Drill JDBC Driver",
- "1.0",
- "Drill",
- "1.0",
- true,
- 1,
- 0,
- 1,
- 0);
- }
-}
-
-
-// End DrillDriverVersion.java
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillFactory.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillFactory.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillFactory.java
new file mode 100644
index 0000000..9b9eb7b
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillFactory.java
@@ -0,0 +1,57 @@
+/**
+ * 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.jdbc;
+
+import java.sql.SQLException;
+import java.util.Properties;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaFactory;
+import net.hydromatic.avatica.UnregisteredDriver;
+
+/**
+ * Extension of {@link net.hydromatic.avatica.AvaticaFactory} for Drill.
+ */
+public abstract class DrillFactory implements AvaticaFactory {
+ protected final int major;
+ protected final int minor;
+
+ /** Creates a JDBC factory with given major/minor version number. */
+ protected DrillFactory(int major, int minor) {
+ this.major = major;
+ this.minor = minor;
+ }
+
+ public int getJdbcMajorVersion() {
+ return major;
+ }
+
+ public int getJdbcMinorVersion() {
+ return minor;
+ }
+
+ public final AvaticaConnection newConnection(UnregisteredDriver driver, AvaticaFactory factory, String url,
+ Properties info) throws SQLException{
+ return newDrillConnection((Driver) driver, (DrillFactory) factory, url, info);
+ }
+
+ /** Creates a connection with a root schema. */
+ public abstract DrillConnectionImpl newDrillConnection(Driver driver, DrillFactory factory, String url,
+ Properties info) throws SQLException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillHandler.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillHandler.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillHandler.java
index 4712302..592bff3 100644
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillHandler.java
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillHandler.java
@@ -17,193 +17,31 @@
*/
package org.apache.drill.jdbc;
-import java.io.IOException;
import java.sql.SQLException;
-import java.util.Map;
-import java.util.Properties;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.optiq.MutableSchema;
-import net.hydromatic.optiq.Schema;
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.impl.java.MapSchema;
-import net.hydromatic.optiq.jdbc.HandlerImpl;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-import net.hydromatic.optiq.model.ModelHandler;
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.Handler;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.coord.ClusterCoordinator;
-import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.store.SchemaProvider;
-import org.apache.drill.exec.store.SchemaProviderRegistry;
-import org.apache.drill.exec.store.hive.HiveStorageEngine.HiveSchemaProvider;
-import org.apache.drill.exec.store.json.JsonSchemaProvider;
-import org.apache.drill.exec.store.parquet.ParquetSchemaProvider;
-import org.apache.drill.sql.client.full.FileSystemSchema;
-
-import com.google.common.base.Charsets;
-import com.google.common.base.Preconditions;
-import com.google.common.io.Resources;
-import org.apache.drill.sql.client.full.HiveSchema;
-
-public class DrillHandler extends HandlerImpl {
+public class DrillHandler implements Handler{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillHandler.class);
- private ClusterCoordinator coordinator;
- private volatile DrillClient client;
- private Drillbit bit;
- private DrillConfig config = DrillConfig.create();
- private SchemaProviderRegistry registry;
- private final boolean ref;
-
- public DrillHandler(boolean ref){
- this.ref = ref;
- }
-
- public void onConnectionInit(OptiqConnection connection) throws SQLException {
- super.onConnectionInit(connection);
-
- final Properties p = connection.getProperties();
-
- if (ref) {
- final String model = p.getProperty("model");
- if (model != null) {
- if (model != null) {
- try {
- new ModelHandler(connection, model);
- } catch (IOException e) {
- throw new SQLException(e);
- }
- }
- }
- } else {
-
- registry = new SchemaProviderRegistry(config);
-
- Preconditions.checkArgument(bit == null);
- Preconditions.checkArgument(client == null);
- Preconditions.checkArgument(coordinator == null);
-
- final String zk = connection.getProperties().getProperty("zk");
-
- try {
- String enginesData = Resources.toString(Resources.getResource("storage-engines.json"), Charsets.UTF_8);
-
- StorageEngines engines = config.getMapper().readValue(enginesData, StorageEngines.class);
-
- if (zk != null) {
- coordinator = new ZKClusterCoordinator(config, zk);
- coordinator.start(10000);
- DrillClient cl = new DrillClient(config, coordinator);
- cl.connect();
- client = cl;
- } else {
-
- RemoteServiceSet local = RemoteServiceSet.getLocalServiceSet();
- this.coordinator = local.getCoordinator();
- bit = new Drillbit(config, local);
- bit.run();
-
- DrillClient cl = new DrillClient(config, coordinator);
- cl.connect();
- client = cl;
- }
-
- MutableSchema rootSchema = connection.getRootSchema();
-
- for (Map.Entry<String, StorageEngineConfig> entry : engines) {
- SchemaProvider provider = registry.getSchemaProvider(entry.getValue());
- Schema schema = getSchema(provider, client, entry.getKey(), entry.getValue(), rootSchema);
- rootSchema.addSchema(entry.getKey(), schema);
- }
-
- rootSchema.addSchema(
- "--FAKE--",
- new FakeSchema(rootSchema, rootSchema.getQueryProvider(), rootSchema.getTypeFactory(), "fake", rootSchema
- .getExpression()));
-
- } catch (Exception ex) {
- System.out.println(ex);
- logger.error("Failure while setting up jdbc handler", ex);
- throw new SQLException("Failure trying to connect to Drill.", ex);
- }
- }
-
- // The "schema" parameter currently gives a name to the schema. In future
- // it will choose a schema that (presumably) already exists.
- final String schemaName = connection.getProperties().getProperty("schema");
- if (schemaName != null) {
- connection.setSchema(schemaName);
- }
-
- final String catalogName = connection.getProperties().getProperty("catalog");
- if (catalogName != null) {
- connection.setCatalog(catalogName);
- }
- }
-
- private Schema getSchema(SchemaProvider provider, DrillClient client, String name, StorageEngineConfig config, Schema rootSchema)
- throws SQLException {
- if (provider instanceof ParquetSchemaProvider || provider instanceof JsonSchemaProvider) {
- return new FileSystemSchema(client, config, provider,
- rootSchema.getTypeFactory(), rootSchema, name, rootSchema.getExpression(),
- rootSchema.getQueryProvider());
- } else if (provider instanceof HiveSchemaProvider) {
- return new HiveSchema(client, config, provider,
- rootSchema.getTypeFactory(), rootSchema, name, rootSchema.getExpression(),
- rootSchema.getQueryProvider());
- }
-
- throw new SQLException("Unknown schema provider");
+ @Override
+ public void onConnectionInit(AvaticaConnection c) throws SQLException {
}
- public class FakeSchema extends MapSchema {
-
- public FakeSchema(Schema parentSchema, QueryProvider queryProvider, JavaTypeFactory typeFactory, String name,
- Expression expression) {
- super(parentSchema, queryProvider, typeFactory, name, expression);
-
- }
-
- public DrillClient getClient() {
- return client;
- }
+ @Override
+ public void onConnectionClose(AvaticaConnection c) throws RuntimeException {
+ DrillConnectionImpl connection = (DrillConnectionImpl) c;
+ connection.cleanup();
}
- public DrillClient getClient() {
- return client;
+ @Override
+ public void onStatementExecute(AvaticaStatement statement, ResultSink resultSink) throws RuntimeException {
}
@Override
- public void onConnectionClose(OptiqConnection connection) throws RuntimeException {
- super.onConnectionClose(connection);
- if (client != null)
- client.close();
- if (bit != null)
- bit.close();
- if (coordinator != null)
- try {
- coordinator.close();
- } catch (IOException e) {
- throw new RuntimeException("Failure closing coordinator.", e);
- }
- bit = null;
- client = null;
- coordinator = null;
+ public void onStatementClose(AvaticaStatement statement) throws RuntimeException {
+ ((DrillRemoteStatement) statement).cleanup();
}
-
}
-/*
- *
- * optiq work ==========
- *
- * 1. todo: test can cast(<any> as varchar) (or indeed to any type)
- *
- * 2. We declare a variant record by adding a '_MAP any' field. It's nice that there can be some declared fields, and
- * some undeclared. todo: Better syntactic sugar.
- */
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java
new file mode 100644
index 0000000..b542eb2
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc40Factory.java
@@ -0,0 +1,29 @@
+/**
+ * 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.jdbc;
+
+/**
+ * Implementation of {@link net.hydromatic.avatica.AvaticaFactory}
+ * for Drill and JDBC 4.0 (corresponds to JDK 1.6).
+ */
+public class DrillJdbc40Factory extends DrillJdbc41Factory {
+ /** Creates a factory for JDBC version 4.1. */
+ public DrillJdbc40Factory() {
+ super(4, 0);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
new file mode 100644
index 0000000..e6e533e
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
@@ -0,0 +1,186 @@
+/**
+ * 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.jdbc;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.sql.NClob;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.util.List;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import net.hydromatic.avatica.AvaticaConnection;
+import net.hydromatic.avatica.AvaticaDatabaseMetaData;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaPreparedStatement;
+import net.hydromatic.avatica.AvaticaResultSetMetaData;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.ColumnMetaData;
+
+/**
+ * Implementation of {@link net.hydromatic.avatica.AvaticaFactory} for Drill and JDBC 4.1 (corresponds to JDK 1.7).
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class DrillJdbc41Factory extends DrillFactory {
+ /** Creates a factory for JDBC version 4.1. */
+ public DrillJdbc41Factory() {
+ this(4, 1);
+ }
+
+ /** Creates a JDBC factory with given major/minor version number. */
+ protected DrillJdbc41Factory(int major, int minor) {
+ super(major, minor);
+ }
+
+ public DrillJdbc41Connection newDrillConnection(Driver driver, DrillFactory factory, String url, Properties info) throws SQLException{
+ return new DrillJdbc41Connection((Driver) driver, factory, url, info);
+ }
+
+ public DrillJdbc41DatabaseMetaData newDatabaseMetaData(AvaticaConnection connection) {
+ return new DrillJdbc41DatabaseMetaData((DrillConnectionImpl) connection);
+ }
+
+ public DrillJdbc41Statement newStatement(AvaticaConnection connection, int resultSetType, int resultSetConcurrency,
+ int resultSetHoldability) {
+ return new DrillJdbc41Statement((DrillConnectionImpl) connection, resultSetType, resultSetConcurrency,
+ resultSetHoldability);
+ }
+
+ public AvaticaPreparedStatement newPreparedStatement(AvaticaConnection connection,
+ AvaticaPrepareResult prepareResult, int resultSetType, int resultSetConcurrency, int resultSetHoldability)
+ throws SQLException {
+ return new DrillJdbc41PreparedStatement((DrillConnectionImpl) connection, (DrillPrepareResult) prepareResult,
+ resultSetType, resultSetConcurrency, resultSetHoldability);
+ }
+
+ public DrillResultSet newResultSet(AvaticaStatement statement, AvaticaPrepareResult prepareResult, TimeZone timeZone) {
+ final ResultSetMetaData metaData = newResultSetMetaData(statement, prepareResult.getColumnList());
+ return new DrillResultSet(statement, (DrillPrepareResult) prepareResult, metaData, timeZone);
+ }
+
+ public ResultSetMetaData newResultSetMetaData(AvaticaStatement statement, List<ColumnMetaData> columnMetaDataList) {
+ return new AvaticaResultSetMetaData(statement, null, columnMetaDataList);
+ }
+
+ private static class DrillJdbc41Connection extends DrillConnectionImpl {
+ DrillJdbc41Connection(Driver driver, DrillFactory factory, String url, Properties info) throws SQLException {
+ super(driver, factory, url, info);
+ }
+
+ }
+
+ private static class DrillJdbc41Statement extends DrillStatement {
+ public DrillJdbc41Statement(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency,
+ int resultSetHoldability) {
+ super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+ }
+ }
+
+ private static class DrillJdbc41PreparedStatement extends DrillPreparedStatement {
+ DrillJdbc41PreparedStatement(DrillConnectionImpl connection, DrillPrepareResult prepareResult, int resultSetType,
+ int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+ super(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
+ }
+
+ public void setRowId(int parameterIndex, RowId x) throws SQLException {
+ getParameter(parameterIndex).setRowId(x);
+ }
+
+ public void setNString(int parameterIndex, String value) throws SQLException {
+ getParameter(parameterIndex).setNString(value);
+ }
+
+ public void setNCharacterStream(int parameterIndex, Reader value, long length) throws SQLException {
+ getParameter(parameterIndex).setNCharacterStream(value, length);
+ }
+
+ public void setNClob(int parameterIndex, NClob value) throws SQLException {
+ getParameter(parameterIndex).setNClob(value);
+ }
+
+ public void setClob(int parameterIndex, Reader reader, long length) throws SQLException {
+ getParameter(parameterIndex).setClob(reader, length);
+ }
+
+ public void setBlob(int parameterIndex, InputStream inputStream, long length) throws SQLException {
+ getParameter(parameterIndex).setBlob(inputStream, length);
+ }
+
+ public void setNClob(int parameterIndex, Reader reader, long length) throws SQLException {
+ getParameter(parameterIndex).setNClob(reader, length);
+ }
+
+ public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
+ getParameter(parameterIndex).setSQLXML(xmlObject);
+ }
+
+ public void setAsciiStream(int parameterIndex, InputStream x, long length) throws SQLException {
+ getParameter(parameterIndex).setAsciiStream(x, length);
+ }
+
+ public void setBinaryStream(int parameterIndex, InputStream x, long length) throws SQLException {
+ getParameter(parameterIndex).setBinaryStream(x, length);
+ }
+
+ public void setCharacterStream(int parameterIndex, Reader reader, long length) throws SQLException {
+ getParameter(parameterIndex).setCharacterStream(reader, length);
+ }
+
+ public void setAsciiStream(int parameterIndex, InputStream x) throws SQLException {
+ getParameter(parameterIndex).setAsciiStream(x);
+ }
+
+ public void setBinaryStream(int parameterIndex, InputStream x) throws SQLException {
+ getParameter(parameterIndex).setBinaryStream(x);
+ }
+
+ public void setCharacterStream(int parameterIndex, Reader reader) throws SQLException {
+ getParameter(parameterIndex).setCharacterStream(reader);
+ }
+
+ public void setNCharacterStream(int parameterIndex, Reader value) throws SQLException {
+ getParameter(parameterIndex).setNCharacterStream(value);
+ }
+
+ public void setClob(int parameterIndex, Reader reader) throws SQLException {
+ getParameter(parameterIndex).setClob(reader);
+ }
+
+ public void setBlob(int parameterIndex, InputStream inputStream) throws SQLException {
+ getParameter(parameterIndex).setBlob(inputStream);
+ }
+
+ public void setNClob(int parameterIndex, Reader reader) throws SQLException {
+ getParameter(parameterIndex).setNClob(reader);
+ }
+ }
+
+ private static class DrillJdbc41DatabaseMetaData extends AvaticaDatabaseMetaData {
+ DrillJdbc41DatabaseMetaData(DrillConnectionImpl connection) {
+ super(connection);
+ }
+ }
+
+}
+
+// End DrillJdbc41Factory.java
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPrepareResult.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPrepareResult.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPrepareResult.java
new file mode 100644
index 0000000..836f346
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPrepareResult.java
@@ -0,0 +1,52 @@
+/**
+ * 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.jdbc;
+
+import java.util.Collections;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaParameter;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.ColumnMetaData;
+
+public class DrillPrepareResult implements AvaticaPrepareResult{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillPrepareResult.class);
+
+ final String sql;
+ final DrillColumnMetaDataList columns = new DrillColumnMetaDataList();
+
+ public DrillPrepareResult(String sql) {
+ super();
+ this.sql = sql;
+ }
+
+ @Override
+ public List<ColumnMetaData> getColumnList() {
+ return columns;
+ }
+
+ @Override
+ public String getSql() {
+ return sql;
+ }
+
+ @Override
+ public List<AvaticaParameter> getParameterList() {
+ return Collections.emptyList();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
new file mode 100644
index 0000000..6d4ee95
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillPreparedStatement.java
@@ -0,0 +1,50 @@
+/**
+ * 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.jdbc;
+
+import java.sql.SQLException;
+
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaPreparedStatement;
+
+/**
+ * Implementation of {@link java.sql.PreparedStatement} for Drill.
+ *
+ * <p>
+ * This class has sub-classes which implement JDBC 3.0 and JDBC 4.0 APIs; it is instantiated using
+ * {@link net.hydromatic.avatica.AvaticaFactory#newPreparedStatement}.
+ * </p>
+ */
+abstract class DrillPreparedStatement extends AvaticaPreparedStatement implements DrillRemoteStatement {
+
+ protected DrillPreparedStatement(DrillConnectionImpl connection, AvaticaPrepareResult prepareResult,
+ int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+ super(connection, prepareResult, resultSetType, resultSetConcurrency, resultSetHoldability);
+ }
+
+ @Override
+ public DrillConnectionImpl getConnection() {
+ return (DrillConnectionImpl) super.getConnection();
+ }
+
+ @Override
+ public void cleanup() {
+ final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
+ connection1.registry.removeStatement(this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
new file mode 100644
index 0000000..e19b49d
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillRemoteStatement.java
@@ -0,0 +1,22 @@
+/**
+ * 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.jdbc;
+
+public interface DrillRemoteStatement {
+ public void cleanup();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
new file mode 100644
index 0000000..be56b96
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
@@ -0,0 +1,196 @@
+/**
+ * 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.jdbc;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.TimeZone;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.ConnectionThrottle;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+
+import com.google.common.collect.Queues;
+
+public class DrillResultSet extends AvaticaResultSet {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillResultSet.class);
+
+ SchemaChangeListener changeListener;
+ final Listener listener = new Listener();
+ private volatile QueryId queryId;
+ private final DrillClient client;
+ final RecordBatchLoader currentBatch;
+ final DrillCursor cursor;
+
+ public DrillResultSet(AvaticaStatement statement, AvaticaPrepareResult prepareResult,
+ ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
+ super(statement, prepareResult, resultSetMetaData, timeZone);
+ DrillConnection c = (DrillConnection) statement.getConnection();
+ DrillClient client = c.getClient();
+ // DrillClient client, DrillStatement statement) {
+ currentBatch = new RecordBatchLoader(client.getAllocator());
+ this.client = client;
+ cursor = new DrillCursor(this);
+ }
+
+ @Override
+ protected void cancel() {
+ cleanup();
+ close();
+ }
+
+ synchronized void cleanup(){
+ if (queryId != null && !listener.completed){
+ client.cancelQuery(queryId);
+ }
+ listener.close();
+ }
+
+ @Override protected DrillResultSet execute() throws SQLException{
+ // Call driver's callback. It is permitted to throw a RuntimeException.
+ DrillConnectionImpl connection = (DrillConnectionImpl) statement.getConnection();
+
+ connection.getClient().runQuery(QueryType.SQL, this.prepareResult.getSql(), listener);
+ connection.getDriver().handler.onStatementExecute(statement, null);
+
+ super.execute();
+
+ // don't return with metadata until we've achieved at least one return message.
+ try {
+ listener.latch.await();
+ cursor.next();
+ } catch (InterruptedException e) {
+ }
+
+ return this;
+ }
+
+ class Listener implements UserResultsListener {
+ private static final int MAX = 100;
+ private volatile RpcException ex;
+ volatile boolean completed = false;
+ private volatile boolean autoread = true;
+ private volatile ConnectionThrottle throttle;
+ private volatile boolean closed = false;
+ private CountDownLatch latch = new CountDownLatch(1);
+ private AtomicBoolean receivedMessage = new AtomicBoolean(false);
+
+
+
+ final LinkedBlockingDeque<QueryResultBatch> queue = Queues.newLinkedBlockingDeque();
+
+ private boolean releaseIfFirst(){
+ if(receivedMessage.compareAndSet(false, true)){
+ latch.countDown();
+ return true;
+ }
+
+ return false;
+ }
+
+ @Override
+ public void submissionFailed(RpcException ex) {
+ releaseIfFirst();
+ this.ex = ex;
+ completed = true;
+ close();
+ System.out.println("Query failed: " + ex);
+ }
+
+ @Override
+ public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+ logger.debug("Result arrived {}", result);
+
+ // if we're in a closed state, just release the message.
+ if (closed) {
+ result.release();
+ completed = true;
+ return;
+ }
+
+ // we're active, let's add to the queue.
+ queue.add(result);
+ if (queue.size() >= MAX - 1) {
+ throttle.setAutoRead(false);
+ this.throttle = throttle;
+ autoread = false;
+ }
+
+ if (result.getHeader().getIsLastChunk()) {
+ completed = true;
+ }
+
+ if (result.getHeader().getErrorCount() > 0) {
+ submissionFailed(new RpcException(String.format("%s", result.getHeader().getErrorList())));
+ }
+
+ releaseIfFirst();
+
+ }
+
+ public QueryResultBatch getNext() throws RpcException, InterruptedException {
+ while (true) {
+ if (ex != null)
+ throw ex;
+ if (completed && queue.isEmpty()) {
+ return null;
+ } else {
+ QueryResultBatch q = queue.poll(50, TimeUnit.MILLISECONDS);
+ if (q != null) {
+ if (!autoread && queue.size() < MAX / 2) {
+ autoread = true;
+ throttle.setAutoRead(true);
+ throttle = null;
+ }
+ return q;
+ }
+
+ }
+
+ }
+ }
+
+ void close() {
+ closed = true;
+ while (!queue.isEmpty()) {
+ QueryResultBatch qrb = queue.poll();
+ if(qrb != null && qrb.getData() != null) qrb.getData().release();
+ }
+ completed = true;
+ }
+
+ @Override
+ public void queryIdArrived(QueryId queryId) {
+ DrillResultSet.this.queryId = queryId;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatement.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatement.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatement.java
new file mode 100644
index 0000000..fec126e
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatement.java
@@ -0,0 +1,39 @@
+/**
+ * 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.jdbc;
+
+import net.hydromatic.avatica.AvaticaStatement;
+
+public abstract class DrillStatement extends AvaticaStatement implements DrillRemoteStatement {
+
+ DrillStatement(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
+ super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+ }
+
+ @Override
+ public DrillConnectionImpl getConnection() {
+ return (DrillConnectionImpl) connection;
+ }
+
+ @Override
+ public void cleanup() {
+ final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
+ connection1.registry.removeStatement(this);
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
new file mode 100644
index 0000000..480caca
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
@@ -0,0 +1,28 @@
+/**
+ * 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.jdbc;
+
+class DrillStatementRegistry {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStatementRegistry.class);
+
+
+ public void addStatement(DrillRemoteStatement statement){}
+ public void removeStatement(DrillRemoteStatement statement){}
+
+ public void close(){}
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
deleted file mode 100644
index f51021e..0000000
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillTable.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/**
- * 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.jdbc;
-
-import java.lang.reflect.Type;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import net.hydromatic.linq4j.BaseQueryable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.MethodCallExpression;
-import net.hydromatic.optiq.BuiltinMethod;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.MutableSchema;
-import net.hydromatic.optiq.Schema;
-import net.hydromatic.optiq.Statistic;
-import net.hydromatic.optiq.Statistics;
-import net.hydromatic.optiq.TableFactory;
-import net.hydromatic.optiq.TranslatableTable;
-
-import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.ref.rops.DataWriter;
-import org.apache.drill.exec.ref.rse.ClasspathRSE;
-import org.apache.drill.exec.ref.rse.ClasspathRSE.ClasspathInputConfig;
-import org.apache.drill.optiq.DrillRel;
-import org.apache.drill.optiq.DrillScan;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.type.SqlTypeName;
-
-/** Optiq Table used by Drill. */
-public class DrillTable extends BaseQueryable<Object> implements TranslatableTable<Object>{
-
- private final Schema schema;
- private final String name;
- private final String storageEngineName;
- private final RelDataType rowType;
- public final StorageEngineConfig storageEngineConfig;
- private Object selection;
- public DrillClient client;
-
-
- public boolean isRefEngine(){
- return client == null;
- }
-
- /** Creates a DrillTable. */
- public DrillTable(DrillClient client, Schema schema,
- Type elementType,
- Expression expression,
- RelDataType rowType,
- String name,
- String storageEngineName,
- Object selection,
- StorageEngineConfig storageEngineConfig
- ) {
- super(schema.getQueryProvider(), elementType, expression);
- this.client = client;
- this.schema = schema;
- this.name = name;
- this.rowType = rowType;
- this.selection = selection;
- this.storageEngineConfig = storageEngineConfig;
- this.storageEngineName = client == null ? storageEngineName : schema.getName();
- }
-
- public String getName() {
- return name;
- }
-
- public StorageEngineConfig getStorageEngineConfig(){
- return storageEngineConfig;
- }
-
- public Object getSelection() {
- return selection;
- }
-
- public static DrillTable createTable(
- DrillClient client,
- RelDataTypeFactory typeFactory,
- Schema schema,
- String name,
- String storageEngineName,
- StorageEngineConfig storageEngineConfig,
- Object selection
- ) {
- final MethodCallExpression call = Expressions.call(schema.getExpression(), //
- BuiltinMethod.DATA_CONTEXT_GET_TABLE.method, //
- Expressions.constant(name), //
- Expressions.constant(Object.class));
-
- final RelDataType rowType =
- typeFactory.createStructType(
- Collections.singletonList(
- typeFactory.createMapType(
- typeFactory.createSqlType(SqlTypeName.VARCHAR),
- typeFactory.createSqlType(SqlTypeName.ANY))),
- Collections.singletonList("_MAP"));
- return new DrillTable(client, schema, Object.class, call, rowType, name, storageEngineName, selection, storageEngineConfig);
- }
-
-
-
- @Override
- public DataContext getDataContext() {
- return schema;
- }
-
-
- public String getStorageEngineName() {
- return storageEngineName;
- }
-
- public boolean useReferenceInterpreter() {
- return this.client == null;
- }
-
- @Override
- public RelDataType getRowType() {
- return rowType;
- }
-
- @Override
- public Statistic getStatistic() {
- return Statistics.UNKNOWN;
- }
-
- @Override
- public Enumerator<Object> enumerator() {
- return Linq4j.emptyEnumerator();
- }
-
- public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable table) {
- return new DrillScan(context.getCluster(),
- context.getCluster().traitSetOf(DrillRel.CONVENTION),
- table);
- }
-
-
-
- /** Factory for custom tables in Optiq schema. */
- @SuppressWarnings("UnusedDeclaration")
- public static class Factory implements TableFactory<DrillTable> {
-
- private static final List<String> DONUTS_TABLES = Arrays.asList(
- "DONUTS");
-
- private static final List<String> HR_TABLES = Arrays.asList(
- "EMPLOYEES", "DEPARTMENTS");
-
- private static final List<String> FOODMART_TABLES = Arrays.asList(
- "ACCOUNT", "CATEGORY", "CURRENCY", "CUSTOMER", "DAYS", "DEPARTMENT",
- "EMPLOYEE_CLOSURE", "EMPLOYEE", "EXPENSE_FACT", "INVENTORY_FACT_1997",
- "INVENTORY_FACT_1998", "POSITION", "PRODUCT_CLASS", "PRODUCT",
- "PROMOTION", "REGION", "RESERVE_EMPLOYEE", "SALARY", "SALES_FACT_1997",
- "SALES_FACT_1998", "SALES_FACT_DEC_1998", "STORE", "STORE_RAGGED",
- "TIME_BY_DAY", "WAREHOUSE", "WAREHOUSE_CLASS");
-
-// public DrillTable create(
-// JavaTypeFactory typeFactory,
-// Schema schema,
-// String name,
-// DistributedMap<String, Object> operand,
-// RelDataType rowType) {
-// final ClasspathRSE.ClasspathRSEConfig rseConfig = new ClasspathRSE.ClasspathRSEConfig();
-// final ClasspathInputConfig inputConfig = new ClasspathInputConfig();
-// assert DONUTS_TABLES.contains(name)
-// || HR_TABLES.contains(name)
-// || FOODMART_TABLES.contains(name)
-// : name;
-// inputConfig.path = "/" + name.toLowerCase() + ".json";
-// inputConfig.type = DataWriter.ConverterType.JSON;
-// boolean useReferenceInterpreter;
-// if (operand.get("useReferenceInterpreter") != null){
-// useReferenceInterpreter = operand.get("useReferenceInterpreter").equals("true") ? true : false;
-// }
-// else{
-// useReferenceInterpreter = false;
-// }
-// return createTable(typeFactory, (MutableSchema) schema, name, rseConfig,
-// inputConfig, "donuts-json", useReferenceInterpreter);
-// }
-//
- @Override
- public DrillTable create(Schema schema, String name, Map<String, Object> operand, RelDataType rowType) {
-
- final ClasspathRSE.ClasspathRSEConfig rseConfig = new ClasspathRSE.ClasspathRSEConfig();
- final ClasspathInputConfig inputConfig = new ClasspathInputConfig();
- assert DONUTS_TABLES.contains(name)
- || HR_TABLES.contains(name)
- || FOODMART_TABLES.contains(name)
- : name;
- inputConfig.path = "/" + name.toLowerCase() + ".json";
- inputConfig.type = DataWriter.ConverterType.JSON;
- return createTable(null, schema.getTypeFactory(), (MutableSchema) schema, name, "donuts-json", rseConfig, inputConfig);
- }
- }
-
-
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java b/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java
index d7878c9..9ac913b 100644
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/Driver.java
@@ -17,55 +17,71 @@
*/
package org.apache.drill.jdbc;
-import net.hydromatic.linq4j.function.Function0;
-import net.hydromatic.optiq.jdbc.DriverVersion;
-import net.hydromatic.optiq.jdbc.Handler;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.UnregisteredDriver;
+import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.optiq.DrillPrepareImpl;
+import net.hydromatic.avatica.DriverVersion;
+import net.hydromatic.avatica.Handler;
+import net.hydromatic.avatica.HandlerImpl;
+import net.hydromatic.avatica.UnregisteredDriver;
/**
- * JDBC driver for Apache Drill.
+ * Optiq JDBC driver.
*/
public class Driver extends UnregisteredDriver {
public static final String CONNECT_STRING_PREFIX = "jdbc:drill:";
- private volatile DrillHandler handler;
+
+
+ final DrillConfig config;
- static {
- new Driver().register();
+ public Driver() {
+ super();
+ config = DrillConfig.create();
}
+
+ public static boolean load(){
+ return true;
+ }
+
+ @Override
protected String getConnectStringPrefix() {
return CONNECT_STRING_PREFIX;
}
- protected DriverVersion createDriverVersion() {
- return new DrillDriverVersion();
+ @Override
+ protected String getFactoryClassName(JdbcVersion jdbcVersion) {
+ switch (jdbcVersion) {
+ case JDBC_30:
+ return "org.apache.drill.jdbc.DrillJdbc3Factory";
+ case JDBC_40:
+ return "org.apache.drill.jdbc.DrillJdbc40Factory";
+ case JDBC_41:
+ default:
+ return "org.apache.drill.jdbc.DrillJdbc41Factory";
+ }
}
- @Override
- protected Function0<OptiqPrepare> createPrepareFactory() {
- return new Function0<OptiqPrepare>() {
- @Override
- public OptiqPrepare apply() {
- return new DrillPrepareImpl(Driver.this);
- }
- };
+ protected DriverVersion createDriverVersion() {
+ return DriverVersion.load(
+ Driver.class,
+ "apache-drill-jdbc.properties",
+ "Drill JDBC Driver",
+ "unknown version",
+ "Optiq",
+ "unknown version");
}
- public DrillClient getClient(){
- return handler.getClient();
+ DrillConfig getConfig(){
+ return config;
}
@Override
protected Handler createHandler() {
- this.handler = new DrillHandler(false);
- return handler;
+ return new HandlerImpl();
}
-
-}
-// End Driver.java
+ static {
+ new Driver().register();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java b/sqlparser/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
new file mode 100644
index 0000000..9326283
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/GlobalServiceSetReference.java
@@ -0,0 +1,29 @@
+/**
+ * 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.jdbc;
+
+import org.apache.drill.exec.server.RemoteServiceSet;
+
+public class GlobalServiceSetReference {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GlobalServiceSetReference.class);
+
+ public static final ThreadLocal<RemoteServiceSet> SETS = new ThreadLocal<RemoteServiceSet>();
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/MetaImpl.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/MetaImpl.java b/sqlparser/src/main/java/org/apache/drill/jdbc/MetaImpl.java
new file mode 100644
index 0000000..5d8b6a7
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/MetaImpl.java
@@ -0,0 +1,180 @@
+/**
+ * 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.jdbc;
+
+import java.sql.ResultSet;
+import java.util.List;
+
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.AvaticaResultSet;
+import net.hydromatic.avatica.AvaticaStatement;
+import net.hydromatic.avatica.Cursor;
+import net.hydromatic.avatica.Meta;
+import net.hydromatic.linq4j.Linq4j;
+
+public class MetaImpl implements Meta {
+
+ static final Driver DRIVER = new Driver();
+
+ final DrillConnectionImpl connection;
+
+ public MetaImpl(DrillConnectionImpl connection) {
+ this.connection = connection;
+ }
+
+ public String getSqlKeywords() {
+ return "";
+ }
+
+ public String getNumericFunctions() {
+ return "";
+ }
+
+ public String getStringFunctions() {
+ return "";
+ }
+
+ public String getSystemFunctions() {
+ return "";
+ }
+
+ public String getTimeDateFunctions() {
+ return "";
+ }
+
+ public static ResultSet getEmptyResultSet() {
+ return null;
+ }
+
+ public ResultSet getTables(String catalog, final Pat schemaPattern, final Pat tableNamePattern,
+ final List<String> typeList) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getSchemas(String catalog, Pat schemaPattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getCatalogs() {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getTableTypes() {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getProcedures(String catalog, Pat schemaPattern, Pat procedureNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getProcedureColumns(String catalog, Pat schemaPattern, Pat procedureNamePattern,
+ Pat columnNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getColumnPrivileges(String catalog, String schema, String table, Pat columnNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getTablePrivileges(String catalog, Pat schemaPattern, Pat tableNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getBestRowIdentifier(String catalog, String schema, String table, int scope, boolean nullable) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getVersionColumns(String catalog, String schema, String table) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getPrimaryKeys(String catalog, String schema, String table) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getImportedKeys(String catalog, String schema, String table) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getExportedKeys(String catalog, String schema, String table) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTable,
+ String foreignCatalog, String foreignSchema, String foreignTable) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getTypeInfo() {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getIndexInfo(String catalog, String schema, String table, boolean unique, boolean approximate) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getUDTs(String catalog, Pat schemaPattern, Pat typeNamePattern, int[] types) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getSuperTypes(String catalog, Pat schemaPattern, Pat typeNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getSuperTables(String catalog, Pat schemaPattern, Pat tableNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getAttributes(String catalog, Pat schemaPattern, Pat typeNamePattern, Pat attributeNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getClientInfoProperties() {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getFunctions(String catalog, Pat schemaPattern, Pat functionNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getFunctionColumns(String catalog, Pat schemaPattern, Pat functionNamePattern, Pat columnNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public ResultSet getPseudoColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+ return getEmptyResultSet();
+ }
+
+ public Cursor createCursor(AvaticaResultSet resultSet_) {
+ return ((DrillResultSet) resultSet_).cursor;
+ }
+
+ public AvaticaPrepareResult prepare(AvaticaStatement statement_, String sql) {
+ //DrillStatement statement = (DrillStatement) statement_;
+ return new DrillPrepareResult(sql);
+ }
+
+ interface Named {
+ String getName();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/RefDriver.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/RefDriver.java b/sqlparser/src/main/java/org/apache/drill/jdbc/RefDriver.java
deleted file mode 100644
index 85c7f5d..0000000
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/RefDriver.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * 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.jdbc;
-
-import net.hydromatic.linq4j.function.Function0;
-import net.hydromatic.optiq.jdbc.DriverVersion;
-import net.hydromatic.optiq.jdbc.Handler;
-import net.hydromatic.optiq.jdbc.OptiqPrepare;
-import net.hydromatic.optiq.jdbc.UnregisteredDriver;
-
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.optiq.DrillPrepareImpl;
-
-/**
- * JDBC driver for Apache Drill.
- */
-public class RefDriver extends UnregisteredDriver {
- public static final String CONNECT_STRING_PREFIX = "jdbc:drillref:";
-
- private volatile DrillHandler handler;
-
- static {
- new RefDriver().register();
- }
-
- protected String getConnectStringPrefix() {
- return CONNECT_STRING_PREFIX;
- }
-
- protected DriverVersion createDriverVersion() {
- return new DrillDriverVersion();
- }
-
- @Override
- protected Function0<OptiqPrepare> createPrepareFactory() {
- return new Function0<OptiqPrepare>() {
- @Override
- public OptiqPrepare apply() {
- return new DrillPrepareImpl(null);
- }
- };
- }
-
- public DrillClient getClient(){
- return handler.getClient();
- }
-
- @Override
- protected Handler createHandler() {
- this.handler = new DrillHandler(true);
- return handler;
- }
-
-}
-
-// End Driver.java
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/SchemaChangeListener.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/SchemaChangeListener.java b/sqlparser/src/main/java/org/apache/drill/jdbc/SchemaChangeListener.java
new file mode 100644
index 0000000..38a39da
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/SchemaChangeListener.java
@@ -0,0 +1,24 @@
+/**
+ * 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.jdbc;
+
+import org.apache.drill.exec.record.BatchSchema;
+
+public interface SchemaChangeListener {
+ public void schemaChanged(BatchSchema newSchema);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java b/sqlparser/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java
new file mode 100644
index 0000000..c24858e
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.jdbc;
+
+import java.sql.SQLException;
+
+/**
+ * Indicates that an operation timed out. This is not an error; you can
+ * retry the operation.
+ */
+public class SqlTimeoutException
+ extends SQLException
+{
+ SqlTimeoutException() {
+ // SQLException(reason, SQLState, vendorCode)
+ // REVIEW mb 19-Jul-05 Is there a standard SQLState?
+ super("timeout", null, 0);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/StorageEngines.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/StorageEngines.java b/sqlparser/src/main/java/org/apache/drill/jdbc/StorageEngines.java
deleted file mode 100644
index d037508..0000000
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/StorageEngines.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * 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.jdbc;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.logical.StorageEngineConfig;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
-
-public class StorageEngines implements Iterable<Map.Entry<String, StorageEngineConfig>>{
-
- private Map<String, StorageEngineConfig> storage;
-
- @JsonCreator
- public StorageEngines(@JsonProperty("storage") Map<String, StorageEngineConfig> storage){
- this.storage = storage;
- }
-
- public static void main(String[] args) throws Exception{
- DrillConfig config = DrillConfig.create();
- String data = Resources.toString(Resources.getResource("storage-engines.json"), Charsets.UTF_8);
- StorageEngines se = config.getMapper().readValue(data, StorageEngines.class);
- System.out.println(se);
- }
-
- @Override
- public String toString() {
- final int maxLen = 10;
- return "StorageEngines [storage=" + (storage != null ? toString(storage.entrySet(), maxLen) : null) + "]";
- }
-
- @Override
- public Iterator<Entry<String, StorageEngineConfig>> iterator() {
- return storage.entrySet().iterator();
- }
-
- private String toString(Collection<?> collection, int maxLen) {
- StringBuilder builder = new StringBuilder();
- builder.append("[");
- int i = 0;
- for (Iterator<?> iterator = collection.iterator(); iterator.hasNext() && i < maxLen; i++) {
- if (i > 0)
- builder.append(", ");
- builder.append(iterator.next());
- }
- builder.append("]");
- return builder.toString();
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b3460af8/sqlparser/src/main/java/org/apache/drill/jdbc/package-info.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/package-info.java b/sqlparser/src/main/java/org/apache/drill/jdbc/package-info.java
new file mode 100644
index 0000000..f465718
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/**
+ * JDBC driver for Drill.
+ */
+package org.apache.drill.jdbc;
|