You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2015/09/18 16:17:40 UTC

svn commit: r1703867 [1/2] - in /lucene/dev/trunk/solr: ./ solrj/src/java/org/apache/solr/client/solrj/io/sql/ solrj/src/resources/ solrj/src/resources/META-INF/ solrj/src/resources/META-INF/services/ solrj/src/test-files/solrj/solr/collection1/conf/ s...

Author: jbernste
Date: Fri Sep 18 14:17:39 2015
New Revision: 1703867

URL: http://svn.apache.org/viewvc?rev=1703867&view=rev
Log:
SOLR-7986: JDBC Driver for SQL Interface

Added:
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ConnectionImpl.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/DriverImpl.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java   (with props)
    lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/package-info.java   (with props)
    lucene/dev/trunk/solr/solrj/src/resources/
    lucene/dev/trunk/solr/solrj/src/resources/META-INF/
    lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/
    lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/java.sql.Driver   (with props)
    lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml   (with props)
    lucene/dev/trunk/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-sql.xml   (with props)
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1703867&r1=1703866&r2=1703867&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Fri Sep 18 14:17:39 2015
@@ -67,6 +67,8 @@ New Features
 
 * SOLR-7903: Add the FacetStream to the Streaming API and wire it into the SQLHandler (Joel Bernstein)
 
+* SOLR-7986: JDBC Driver for SQL Interface (Uwe Schindler, Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-7876: Speed up queries and operations that use many terms when timeAllowed has not been

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ConnectionImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ConnectionImpl.java?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ConnectionImpl.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ConnectionImpl.java Fri Sep 18 14:17:39 2015
@@ -0,0 +1,333 @@
+package org.apache.solr.client.solrj.io.sql;
+
+/*
+ * 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.
+ */
+
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executor;
+
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.io.SolrClientCache;
+
+class ConnectionImpl implements Connection {
+
+  private SolrClientCache sqlSolrClientCache = new SolrClientCache();
+  private CloudSolrClient client;
+  private String collection;
+  Properties props;
+  private boolean closed;
+
+  ConnectionImpl(String zkHost, String collection, Properties props) {
+    this.client = sqlSolrClientCache.getCloudSolrClient(zkHost);
+    this.collection = collection;
+    this.props = props;
+  }
+
+  @Override
+  public Statement createStatement() throws SQLException {
+    return new StatementImpl(client, this.collection, props, sqlSolrClientCache);
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql) throws SQLException {
+    return null;
+  }
+
+  @Override
+  public CallableStatement prepareCall(String sql) throws SQLException {
+    return null;
+  }
+
+  @Override
+  public String nativeSQL(String sql) throws SQLException {
+    return null;
+  }
+
+  @Override
+  public void setAutoCommit(boolean autoCommit) throws SQLException {
+
+  }
+
+  @Override
+  public boolean getAutoCommit() throws SQLException {
+    return false;
+  }
+
+  @Override
+  public void commit() throws SQLException {
+
+  }
+
+  @Override
+  public void rollback() throws SQLException {
+
+  }
+
+  @Override
+  public void close() throws SQLException {
+    if(closed) {
+      return;
+    }
+    try {
+      this.sqlSolrClientCache.close();
+      this.closed = true;
+    } catch (Exception e) {
+      throw new SQLException(e);
+    }
+  }
+
+  @Override
+  public boolean isClosed() throws SQLException {
+    return closed;
+  }
+
+  @Override
+  public DatabaseMetaData getMetaData() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setReadOnly(boolean readOnly) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isReadOnly() throws SQLException {
+    return true;
+  }
+
+  @Override
+  public void setCatalog(String catalog) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getCatalog() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setTransactionIsolation(int level) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getTransactionIsolation() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLWarning getWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Map<String, Class<?>> getTypeMap() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setHoldability(int holdability) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getHoldability() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Savepoint setSavepoint() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Savepoint setSavepoint(String name) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void rollback(Savepoint savepoint) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Clob createClob() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Blob createBlob() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public NClob createNClob() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLXML createSQLXML() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isValid(int timeout) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setClientInfo(String name, String value) throws SQLClientInfoException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setClientInfo(Properties properties) throws SQLClientInfoException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getClientInfo(String name) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Properties getClientInfo() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setSchema(String schema) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getSchema() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void abort(Executor executor) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getNetworkTimeout() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/DriverImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/DriverImpl.java?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/DriverImpl.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/DriverImpl.java Fri Sep 18 14:17:39 2015
@@ -0,0 +1,148 @@
+package org.apache.solr.client.solrj.io.sql;
+
+/*
+ * 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.
+ */
+
+
+import java.net.URLDecoder;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.util.Properties;
+import java.util.logging.Logger;
+import org.apache.solr.common.util.SuppressForbidden;
+
+/**
+ *  Get a Connection with with a url and properties.
+ *
+ *  jdbc:solr://zkhost:port?collection=collection&amp;aggregationMode=map_reduce
+ **/
+
+
+public class DriverImpl implements Driver {
+
+  static {
+    try {
+      DriverManager.registerDriver(new DriverImpl());
+    } catch (SQLException e) {
+      throw new RuntimeException("Can't register driver!", e);
+    }
+  }
+
+  public Connection connect(String url, Properties props) throws SQLException {
+    if(!acceptsURL(url)) {
+      return null;
+    }
+
+    StringBuilder buf = new StringBuilder(url);
+    boolean needsAmp = true;
+    if(!url.contains("?")) {
+      buf.append("?");
+      needsAmp = false;
+    }
+
+    for(Object key : props.keySet()) {
+      Object value = props.get(key);
+      if(needsAmp) {
+        buf.append("&");
+      }
+      buf.append(key.toString()).append("=").append(value);
+      needsAmp = true;
+    }
+
+    return connect(buf.toString());
+  }
+
+  public Connection connect(String url) throws SQLException {
+
+    if(!acceptsURL(url)) {
+      return null;
+    }
+
+    String[] parts = url.split("://", 0);
+
+    if(parts.length < 2) {
+      throw new SQLException("The zkHost must start with ://");
+    }
+
+    String zkUrl  = parts[1];
+    String[] zkUrlParts = zkUrl.split("\\?");
+
+    if(zkUrlParts.length < 2) {
+      throw new SQLException("The connection url has no connection properties. At a mininum the collection must be specified.");
+    }
+
+    String connectionProps = zkUrlParts[1];
+    String zkHost = zkUrlParts[0];
+    Properties props = new Properties();
+    loadParams(connectionProps, props);
+    String collection = (String)props.remove("collection");
+
+    if(!props.containsKey("aggregationMode")) {
+      props.setProperty("aggregationMode","facet");
+    }
+
+    return new ConnectionImpl(zkHost, collection, props);
+  }
+
+  public int getMajorVersion() {
+    return 1;
+  }
+
+  public int getMinorVersion() {
+    return 0;
+  }
+
+  public boolean acceptsURL(String url) {
+    if(url.startsWith("jdbc:solr")) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public boolean jdbcCompliant() {
+    return false;
+  }
+
+
+  @SuppressForbidden(reason="Required by jdbc")
+
+  public Logger getParentLogger() {
+    return null;
+  }
+
+  public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) {
+    return null;
+  }
+
+  private void loadParams(String params, Properties props) throws SQLException {
+    try {
+      String[] pairs = params.split("&");
+      for (String pair : pairs) {
+        String[] keyValue = pair.split("=");
+        String key = URLDecoder.decode(keyValue[0], "UTF-8");
+        String value = URLDecoder.decode(keyValue[1], "UTF-8");
+        props.put(key, value);
+      }
+    } catch(Exception e) {
+      throw new SQLException(e);
+    }
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/ResultSetImpl.java Fri Sep 18 14:17:39 2015
@@ -0,0 +1,1026 @@
+package org.apache.solr.client.solrj.io.sql;
+
+/*
+ * 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.
+ */
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.Ref;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.RowId;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.io.stream.SolrStream;
+import org.apache.solr.client.solrj.io.Tuple;
+
+class ResultSetImpl implements ResultSet {
+
+  private SolrStream solrStream;
+  private Tuple tuple;
+  private boolean done;
+  private boolean closed;
+
+  ResultSetImpl(SolrStream solrStream) {
+    this.solrStream = solrStream;
+  }
+
+  @Override
+  public boolean next() throws SQLException {
+    try {
+
+      if(done) {
+        return false;
+      }
+
+      tuple = solrStream.read();
+      if(tuple.EOF) {
+        done = true;
+        return false;
+      } else {
+        return true;
+      }
+    } catch (Exception e) {
+      throw new SQLException(e);
+    }
+  }
+
+  @Override
+  public void close() throws SQLException {
+    this.done = this.closed = true;
+  }
+
+  @Override
+  public boolean wasNull() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getString(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean getBoolean(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte getByte(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public short getShort(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getInt(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getLong(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public float getFloat(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double getDouble(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex, int scale) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte[] getBytes(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Date getDate(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Time getTime(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Timestamp getTimestamp(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getAsciiStream(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getUnicodeStream(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getBinaryStream(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getString(String columnLabel) throws SQLException {
+    return tuple.getString(columnLabel);
+  }
+
+  @Override
+  public boolean getBoolean(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte getByte(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public short getShort(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getInt(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getLong(String columnLabel) throws SQLException {
+    return tuple.getLong(columnLabel);
+  }
+
+  @Override
+  public float getFloat(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double getDouble(String columnLabel) throws SQLException {
+    return tuple.getDouble(columnLabel);
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnLabel, int scale) throws SQLException {
+    return null;
+  }
+
+  @Override
+  public byte[] getBytes(String columnLabel) throws SQLException {
+    return new byte[0];
+  }
+
+  @Override
+  public Date getDate(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Time getTime(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Timestamp getTimestamp(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getAsciiStream(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getUnicodeStream(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public InputStream getBinaryStream(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLWarning getWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getCursorName() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ResultSetMetaData getMetaData() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object getObject(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object getObject(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int findColumn(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Reader getCharacterStream(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Reader getCharacterStream(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BigDecimal getBigDecimal(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isBeforeFirst() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isAfterLast() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isFirst() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isLast() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void beforeFirst() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void afterLast() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean first() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean last() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean absolute(int row) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean relative(int rows) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean previous() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setFetchDirection(int direction) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getFetchDirection() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setFetchSize(int rows) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getFetchSize() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getType() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getConcurrency() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean rowUpdated() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean rowInserted() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean rowDeleted() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNull(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBoolean(int columnIndex, boolean x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateByte(int columnIndex, byte x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateShort(int columnIndex, short x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateInt(int columnIndex, int x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateLong(int columnIndex, long x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateFloat(int columnIndex, float x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateDouble(int columnIndex, double x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateString(int columnIndex, String x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBytes(int columnIndex, byte[] x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateDate(int columnIndex, Date x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateTime(int columnIndex, Time x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(int columnIndex, InputStream x, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(int columnIndex, Reader x, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateObject(int columnIndex, Object x, int scaleOrLength) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateObject(int columnIndex, Object x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNull(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBoolean(String columnLabel, boolean x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateByte(String columnLabel, byte x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateShort(String columnLabel, short x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateInt(String columnLabel, int x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateLong(String columnLabel, long x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateFloat(String columnLabel, float x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateDouble(String columnLabel, double x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBigDecimal(String columnLabel, BigDecimal x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateString(String columnLabel, String x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBytes(String columnLabel, byte[] x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateDate(String columnLabel, Date x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateTime(String columnLabel, Time x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(String columnLabel, InputStream x, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(String columnLabel, InputStream x, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(String columnLabel, Reader reader, int length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x, int scaleOrLength) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void insertRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void deleteRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void refreshRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void cancelRowUpdates() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void moveToInsertRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void moveToCurrentRow() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Statement getStatement() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object getObject(int columnIndex, Map<String, Class<?>> map) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Ref getRef(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Blob getBlob(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Clob getClob(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Array getArray(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object getObject(String columnLabel, Map<String, Class<?>> map) throws SQLException {
+    throw new UnsupportedOperationException();
+
+  }
+
+  @Override
+  public Ref getRef(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Blob getBlob(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Clob getClob(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Array getArray(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Date getDate(int columnIndex, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Date getDate(String columnLabel, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Time getTime(int columnIndex, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Time getTime(String columnLabel, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Timestamp getTimestamp(int columnIndex, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Timestamp getTimestamp(String columnLabel, Calendar cal) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public URL getURL(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public URL getURL(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateRef(int columnIndex, Ref x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateRef(String columnLabel, Ref x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(int columnIndex, Blob x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(String columnLabel, Blob x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(int columnIndex, Clob x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(String columnLabel, Clob x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateArray(int columnIndex, Array x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateArray(String columnLabel, Array x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public RowId getRowId(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public RowId getRowId(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateRowId(int columnIndex, RowId x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateRowId(String columnLabel, RowId x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getHoldability() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isClosed() throws SQLException {
+    return closed;
+  }
+
+  @Override
+  public void updateNString(int columnIndex, String nString) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNString(String columnLabel, String nString) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(int columnIndex, NClob nClob) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(String columnLabel, NClob nClob) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public NClob getNClob(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public NClob getNClob(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLXML getSQLXML(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLXML getSQLXML(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateSQLXML(int columnIndex, SQLXML xmlObject) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateSQLXML(String columnLabel, SQLXML xmlObject) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getNString(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getNString(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Reader getNCharacterStream(int columnIndex) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Reader getNCharacterStream(String columnLabel) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(int columnIndex, InputStream x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(int columnIndex, InputStream x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(int columnIndex, Reader x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(String columnLabel, InputStream x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(String columnLabel, InputStream x, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(String columnLabel, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(int columnIndex, InputStream inputStream, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(String columnLabel, InputStream inputStream, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(int columnIndex, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(String columnLabel, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(int columnIndex, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(String columnLabel, Reader reader, long length) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNCharacterStream(int columnIndex, Reader x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNCharacterStream(String columnLabel, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(int columnIndex, InputStream x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(int columnIndex, InputStream x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(int columnIndex, Reader x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateAsciiStream(String columnLabel, InputStream x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBinaryStream(String columnLabel, InputStream x) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateCharacterStream(String columnLabel, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(int columnIndex, InputStream inputStream) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateBlob(String columnLabel, InputStream inputStream) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(int columnIndex, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateClob(String columnLabel, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(int columnIndex, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void updateNClob(String columnLabel, Reader reader) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T getObject(int columnIndex, Class<T> type) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T getObject(String columnLabel, Class<T> type) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/StatementImpl.java Fri Sep 18 14:17:39 2015
@@ -0,0 +1,339 @@
+package org.apache.solr.client.solrj.io.sql;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLWarning;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.solr.client.solrj.io.stream.SolrStream;
+import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.io.stream.StreamContext;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+
+class StatementImpl implements Statement {
+
+  private CloudSolrClient client;
+  private SolrClientCache sqlSolrClientCache;
+  private String collection;
+  private Properties properties;
+  private SolrStream solrStream;
+  private boolean closed;
+
+  StatementImpl(CloudSolrClient client, String collection, Properties properties, SolrClientCache sqlSolrClientCache) {
+    this.client = client;
+    this.collection = collection;
+    this.properties = properties;
+    this.sqlSolrClientCache = sqlSolrClientCache;
+  }
+
+  @Override
+  public ResultSet executeQuery(String sql) throws SQLException {
+
+    try {
+      closed = false;  // If closed reopen so Statement can be reused.
+      this.solrStream = constructStream(sql);
+      StreamContext context = new StreamContext();
+      context.setSolrClientCache(sqlSolrClientCache);
+      this.solrStream.setStreamContext(context);
+      this.solrStream.open();
+      return new ResultSetImpl(this.solrStream);
+    } catch(Exception e) {
+      throw new SQLException(e);
+    }
+  }
+
+  protected SolrStream constructStream(String sql) throws IOException {
+
+    try {
+      ZkStateReader zkStateReader = client.getZkStateReader();
+      ClusterState clusterState = zkStateReader.getClusterState();
+      Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
+
+      if(slices == null) {
+        throw new Exception("Collection not found:"+this.collection);
+      }
+
+      Map params = new HashMap();
+
+      List<Replica> shuffler = new ArrayList();
+      for(Slice slice : slices) {
+        Collection<Replica> replicas = slice.getReplicas();
+        for (Replica replica : replicas) {
+          shuffler.add(replica);
+        }
+      }
+
+      Collections.shuffle(shuffler, new Random());
+
+      params.put(CommonParams.QT, "/sql");
+      params.put("sql", sql);
+      params.putAll(properties);
+
+      Replica rep = shuffler.get(0);
+      ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
+      String url = zkProps.getCoreUrl();
+      return new SolrStream(url, params);
+
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public int executeUpdate(String sql) throws SQLException {
+    return 0;
+  }
+
+  @Override
+  public void close() throws SQLException {
+
+    if(closed) {
+      return;
+    }
+
+    try {
+      this.solrStream.close();
+      this.closed = true;
+    } catch (Exception e) {
+      throw new SQLException(e);
+    }
+  }
+
+  @Override
+  public int getMaxFieldSize() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setMaxFieldSize(int max) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getMaxRows() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setMaxRows(int max) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setEscapeProcessing(boolean enable) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getQueryTimeout() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setQueryTimeout(int seconds) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void cancel() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SQLWarning getWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clearWarnings() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setCursorName(String name) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean execute(String sql) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ResultSet getResultSet() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getUpdateCount() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean getMoreResults() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setFetchDirection(int direction) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getFetchDirection() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setFetchSize(int rows) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getFetchSize() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getResultSetConcurrency() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getResultSetType() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void addBatch(String sql) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clearBatch() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int[] executeBatch() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Connection getConnection() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean getMoreResults(int current) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ResultSet getGeneratedKeys() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean execute(String sql, int[] columnIndexes) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean execute(String sql, String[] columnNames) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getResultSetHoldability() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isClosed() throws SQLException {
+    return closed;
+  }
+
+  @Override
+  public void setPoolable(boolean poolable) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isPoolable() throws SQLException {
+    return true;
+  }
+
+  @Override
+  public void closeOnCompletion() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isCloseOnCompletion() throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T unwrap(Class<T> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isWrapperFor(Class<?> iface) throws SQLException {
+    throw new UnsupportedOperationException();
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/package-info.java?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/package-info.java (added)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/io/sql/package-info.java Fri Sep 18 14:17:39 2015
@@ -0,0 +1,51 @@
+/*
+ * 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 Package
+ *
+ * Sample usage
+ * <pre>
+ * Connection con = null;
+ * Statement stmt = null;
+ * ResultSet rs = null;
+ *
+ * try {
+ *  con = DriverManager.getConnection("jdbc:solr://zkHost:port?collection=collection&amp;aggregationMode=map_reduce");
+ *  stmt = con.createStatement();
+ *  rs = stmt.executeQuery("select a, sum(b) from tablex group by a");
+ *  while(rs.next()) {
+ *    String a = rs.getString("a");
+ *    double sumB = rs.getString("sum(b)");
+ *  }
+ * } finally {
+ *  rs.close();
+ *  stmt.close();
+ *  con.close();
+ * }
+ * </pre>
+ *
+ * Connection properties can also be passed in using a Properties object.
+ *
+ * The <b>collection</b> parameter is mandatory and should point to a SolrCloud collection that is configured with the /sql
+ * request handler.
+ *
+ * The aggregationMode parameter is optional. It can be used to switch between Map/Reduce (map_reduce) or the JSON Facet API (facet) for
+ * group by aggregations. The default is "facet".
+ **/
+
+package org.apache.solr.client.solrj.io.sql;

Added: lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/java.sql.Driver
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/java.sql.Driver?rev=1703867&view=auto
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/java.sql.Driver (added)
+++ lucene/dev/trunk/solr/solrj/src/resources/META-INF/services/java.sql.Driver Fri Sep 18 14:17:39 2015
@@ -0,0 +1,16 @@
+#  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.
+
+org.apache.solr.client.solrj.io.sql.DriverImpl