mirror of https://github.com/apache/lucene.git
Fix JDBC tests, update dependencies, integrate Calcite more
This commit is contained in:
parent
e554ae404c
commit
52cc4d16bf
|
@ -133,9 +133,9 @@
|
|||
<dependency org="com.tdunning" name="t-digest" rev="${/com.tdunning/t-digest}" conf="compile->*"/>
|
||||
|
||||
<!-- SQL Parser -->
|
||||
<dependency org="org.apache.calcite" name="calcite-core" rev="1.6.0"/>
|
||||
<dependency org="org.apache.calcite" name="calcite-linq4j" rev="1.6.0"/>
|
||||
<dependency org="org.apache.calcite" name="calcite-avatica" rev="1.6.0"/>
|
||||
<dependency org="org.apache.calcite" name="calcite-core" rev="1.7.0"/>
|
||||
<dependency org="org.apache.calcite" name="calcite-linq4j" rev="1.7.0"/>
|
||||
<dependency org="org.apache.calcite.avatica" name="avatica" rev="1.7.1"/>
|
||||
<dependency org="net.hydromatic" name="eigenbase-properties" rev="1.1.5"/>
|
||||
<dependency org="org.codehaus.janino" name="janino" rev="2.7.6"/>
|
||||
<dependency org="org.codehaus.janino" name="commons-compiler" rev="2.7.6"/>
|
||||
|
|
|
@ -16,19 +16,36 @@
|
|||
*/
|
||||
package org.apache.solr.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.solr.client.solrj.io.Tuple;
|
||||
import org.apache.solr.client.solrj.io.comp.StreamComparator;
|
||||
import org.apache.solr.client.solrj.io.stream.ExceptionStream;
|
||||
import org.apache.solr.client.solrj.io.stream.JDBCStream;
|
||||
import org.apache.solr.client.solrj.io.stream.StreamContext;
|
||||
import org.apache.solr.client.solrj.io.stream.TupleStream;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.Explanation;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.sql.SolrSchemaFactory;
|
||||
import org.apache.solr.handler.sql.CalciteSolrDriver;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.security.AuthorizationContext;
|
||||
|
@ -60,10 +77,15 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
}
|
||||
|
||||
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
|
||||
SolrParams params = req.getParams();
|
||||
params = adjustParams(params);
|
||||
req.setParams(params);
|
||||
ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
|
||||
String sql = params.get("stmt");
|
||||
// Set defaults for parameters
|
||||
params.set("numWorkers", params.getInt("numWorkers", 1));
|
||||
params.set("workerCollection", params.get("workerCollection", defaultWorkerCollection));
|
||||
params.set("workerZkhost", params.get("workerZkhost", defaultZkhost));
|
||||
params.set("aggregationMode", params.get("aggregationMode", "map_reduce"));
|
||||
// JDBC driver requires metadata from the SQLHandler. Default to false since this adds a new Metadata stream.
|
||||
params.set("includeMetadata", params.getBool("includeMetadata", false));
|
||||
|
||||
TupleStream tupleStream = null;
|
||||
try {
|
||||
|
@ -71,26 +93,11 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
throw new Exception("stmt parameter cannot be null");
|
||||
}
|
||||
|
||||
Properties info = new Properties();
|
||||
info.setProperty("model",
|
||||
"inline:{\n" +
|
||||
" \"version\": \"1.0\",\n" +
|
||||
" \"defaultSchema\": \"" + defaultZkhost + "\",\n" +
|
||||
" \"schemas\": [\n" +
|
||||
" {\n" +
|
||||
" \"name\": \"" + defaultZkhost + "\",\n" +
|
||||
" \"type\": \"custom\",\n" +
|
||||
" \"factory\": \"" + SolrSchemaFactory.class.getName() + "\",\n" +
|
||||
" \"operand\": {\n" +
|
||||
" \"zk\": \"" + defaultZkhost + "\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
" ]\n" +
|
||||
"}");
|
||||
info.setProperty("lex", "MYSQL");
|
||||
|
||||
tupleStream = new StreamHandler.TimerStream(new ExceptionStream(
|
||||
new JDBCStream("jdbc:calcite:", sql, null, info, null)));
|
||||
/*
|
||||
* Would be great to replace this with the JDBCStream. Can't do that currently since need to have metadata
|
||||
* added to the stream for the JDBC driver. This could be fixed by using the Calcite Avatica server and client.
|
||||
*/
|
||||
tupleStream = new StreamHandler.TimerStream(new ExceptionStream(new SqlHandlerStream(sql, params)));
|
||||
|
||||
rsp.add("result-set", tupleStream);
|
||||
} catch(Exception e) {
|
||||
|
@ -103,13 +110,6 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
}
|
||||
}
|
||||
|
||||
private SolrParams adjustParams(SolrParams params) {
|
||||
ModifiableSolrParams adjustedParams = new ModifiableSolrParams();
|
||||
adjustedParams.add(params);
|
||||
adjustedParams.add(CommonParams.OMIT_HEADER, "true");
|
||||
return adjustedParams;
|
||||
}
|
||||
|
||||
public String getDescription() {
|
||||
return "SQLHandler";
|
||||
}
|
||||
|
@ -117,4 +117,125 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
public String getSource() {
|
||||
return null;
|
||||
}
|
||||
|
||||
private class SqlHandlerStream extends TupleStream {
|
||||
private final String sql;
|
||||
private final SolrParams params;
|
||||
private boolean firstTuple = true;
|
||||
private Connection connection;
|
||||
private Statement statement;
|
||||
private ResultSet resultSet;
|
||||
private ResultSetMetaData resultSetMetaData;
|
||||
private int numColumns;
|
||||
|
||||
SqlHandlerStream(String sql, SolrParams params) {
|
||||
this.sql = sql;
|
||||
this.params = params;
|
||||
}
|
||||
|
||||
public List<TupleStream> children() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
public void open() throws IOException {
|
||||
Properties properties = new Properties();
|
||||
// Add all query parameters
|
||||
Iterator<String> parameterNamesIterator = params.getParameterNamesIterator();
|
||||
while(parameterNamesIterator.hasNext()) {
|
||||
String param = parameterNamesIterator.next();
|
||||
properties.setProperty(param, params.get(param));
|
||||
}
|
||||
|
||||
// Set these last to ensure that they are set properly
|
||||
properties.setProperty("lex", "MYSQL");
|
||||
properties.setProperty("zk", defaultZkhost);
|
||||
|
||||
try {
|
||||
Class.forName(CalciteSolrDriver.class.getCanonicalName());
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
try {
|
||||
connection = DriverManager.getConnection("jdbc:calcitesolr:", properties);
|
||||
statement = connection.createStatement();
|
||||
resultSet = statement.executeQuery(sql);
|
||||
resultSetMetaData = this.resultSet.getMetaData();
|
||||
numColumns = resultSetMetaData.getColumnCount();
|
||||
} catch (SQLException e) {
|
||||
this.close();
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation toExplanation(StreamFactory factory) throws IOException {
|
||||
|
||||
return new StreamExplanation(getStreamNodeId().toString())
|
||||
.withFunctionName("SQL Handler")
|
||||
.withExpression("--non-expressible--")
|
||||
.withImplementingClass(this.getClass().getName())
|
||||
.withExpressionType(Explanation.ExpressionType.STREAM_DECORATOR);
|
||||
}
|
||||
|
||||
// Return a metadata tuple as the first tuple and then pass through to the underlying stream.
|
||||
public Tuple read() throws IOException {
|
||||
try {
|
||||
Map<String, Object> fields = new HashMap<>();
|
||||
if(firstTuple && params.getBool("includeMetadata")) {
|
||||
firstTuple = false;
|
||||
|
||||
List<String> metadataFields = new ArrayList<>();
|
||||
Map<String, String> metadataAliases = new HashMap<>();
|
||||
for(int i = 1; i <= numColumns; i++) {
|
||||
String columnName = resultSetMetaData.getColumnName(i);
|
||||
String columnLabel = resultSetMetaData.getColumnLabel(i);
|
||||
metadataFields.add(columnName);
|
||||
metadataAliases.put(columnName, columnLabel);
|
||||
}
|
||||
|
||||
fields.put("isMetadata", true);
|
||||
fields.put("fields", metadataFields);
|
||||
fields.put("aliases", metadataAliases);
|
||||
} else {
|
||||
if(this.resultSet.next()){
|
||||
for(int i = 1; i <= numColumns; i++) {
|
||||
fields.put(resultSetMetaData.getColumnName(i), this.resultSet.getObject(i));
|
||||
}
|
||||
} else {
|
||||
fields.put("EOF", true);
|
||||
}
|
||||
|
||||
}
|
||||
return new Tuple(fields);
|
||||
} catch (SQLException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public StreamComparator getStreamSort() {
|
||||
return null;
|
||||
}
|
||||
|
||||
private void closeQuietly(AutoCloseable closeable) {
|
||||
if(closeable != null) {
|
||||
try {
|
||||
closeable.close();
|
||||
} catch (Exception ignore) {
|
||||
} finally {
|
||||
closeable = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
this.closeQuietly(this.resultSet);
|
||||
this.closeQuietly(this.statement);
|
||||
this.closeQuietly(this.connection);
|
||||
}
|
||||
|
||||
public void setStreamContext(StreamContext context) {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* 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.solr.handler.sql;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.SQLException;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.calcite.jdbc.CalciteConnection;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
|
||||
/**
|
||||
* JDBC driver for Calcite Solr.
|
||||
*
|
||||
* <p>It accepts connect strings that start with "jdbc:calcitesolr:".</p>
|
||||
*/
|
||||
public class CalciteSolrDriver extends org.apache.calcite.jdbc.Driver {
|
||||
protected CalciteSolrDriver() {
|
||||
super();
|
||||
}
|
||||
|
||||
static {
|
||||
new CalciteSolrDriver().register();
|
||||
}
|
||||
|
||||
protected String getConnectStringPrefix() {
|
||||
return "jdbc:calcitesolr:";
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Connection connect(String url, Properties info) throws SQLException {
|
||||
Connection connection = super.connect(url, info);
|
||||
CalciteConnection calciteConnection = (CalciteConnection) connection;
|
||||
final SchemaPlus rootSchema = calciteConnection.getRootSchema();
|
||||
|
||||
String schemaName = info.getProperty("zk");
|
||||
rootSchema.add(schemaName, new SolrSchema(info));
|
||||
|
||||
// Set the default schema
|
||||
calciteConnection.setSchema(schemaName);
|
||||
|
||||
return connection;
|
||||
}
|
||||
}
|
|
@ -36,6 +36,11 @@ class SolrEnumerator implements Enumerator<Object> {
|
|||
*/
|
||||
SolrEnumerator(TupleStream tupleStream, List<String> fields) {
|
||||
this.tupleStream = tupleStream;
|
||||
try {
|
||||
this.tupleStream.open();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
this.fields = fields;
|
||||
this.current = null;
|
||||
}
|
||||
|
@ -82,7 +87,7 @@ class SolrEnumerator implements Enumerator<Object> {
|
|||
try {
|
||||
this.tupleStream.close();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.solr.handler.sql;
|
|||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -37,15 +38,16 @@ import org.apache.solr.client.solrj.response.LukeResponse;
|
|||
import org.apache.solr.common.luke.FieldFlag;
|
||||
|
||||
class SolrSchema extends AbstractSchema {
|
||||
final String zk;
|
||||
final Properties properties;
|
||||
|
||||
SolrSchema(String zk) {
|
||||
SolrSchema(Properties properties) {
|
||||
super();
|
||||
this.zk = zk;
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, Table> getTableMap() {
|
||||
String zk = this.properties.getProperty("zk");
|
||||
try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zk).build()) {
|
||||
cloudSolrClient.connect();
|
||||
Set<String> collections = cloudSolrClient.getZkStateReader().getClusterState().getCollections();
|
||||
|
@ -61,6 +63,7 @@ class SolrSchema extends AbstractSchema {
|
|||
}
|
||||
|
||||
private Map<String, LukeResponse.FieldInfo> getFieldInfo(String collection) {
|
||||
String zk = this.properties.getProperty("zk");
|
||||
try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder().withZkHost(zk).build()) {
|
||||
cloudSolrClient.connect();
|
||||
LukeRequest lukeRequest = new LukeRequest();
|
||||
|
|
|
@ -1,34 +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.solr.handler.sql;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.calcite.schema.Schema;
|
||||
import org.apache.calcite.schema.SchemaFactory;
|
||||
import org.apache.calcite.schema.SchemaPlus;
|
||||
|
||||
@SuppressWarnings("UnusedDeclaration")
|
||||
public class SolrSchemaFactory implements SchemaFactory {
|
||||
public SolrSchemaFactory() {
|
||||
}
|
||||
|
||||
public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
|
||||
final String zk = (String) operand.get("zk");
|
||||
return new SolrSchema(zk);
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.calcite.adapter.java.AbstractQueryableTable;
|
||||
import org.apache.calcite.linq4j.AbstractEnumerable;
|
||||
|
@ -37,7 +38,6 @@ import org.apache.calcite.rel.type.RelProtoDataType;
|
|||
import org.apache.calcite.schema.SchemaPlus;
|
||||
import org.apache.calcite.schema.TranslatableTable;
|
||||
import org.apache.calcite.schema.impl.AbstractTableQueryable;
|
||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||
import org.apache.solr.client.solrj.io.stream.CloudSolrStream;
|
||||
import org.apache.solr.client.solrj.io.stream.TupleStream;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
|
@ -69,20 +69,21 @@ public class SolrTable extends AbstractQueryableTable implements TranslatableTab
|
|||
return protoRowType.apply(typeFactory);
|
||||
}
|
||||
|
||||
public Enumerable<Object> query(final String zk) {
|
||||
return query(zk, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), null);
|
||||
public Enumerable<Object> query(final Properties properties) {
|
||||
return query(properties, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), null);
|
||||
}
|
||||
|
||||
/** Executes a Solr query on the underlying table.
|
||||
*
|
||||
* @param zk Solr ZooKeeper connection string
|
||||
* @param properties Connections properties
|
||||
* @param fields List of fields to project
|
||||
* @param filterQueries A list of filterQueries which should be used in the query
|
||||
* @return Enumerator of results
|
||||
*/
|
||||
public Enumerable<Object> query(final String zk, List<String> fields,
|
||||
public Enumerable<Object> query(final Properties properties, List<String> fields,
|
||||
List<String> filterQueries, List<String> order, String limit) {
|
||||
Map<String, String> solrParams = new HashMap<>();
|
||||
//solrParams.put(CommonParams.OMIT_HEADER, "true");
|
||||
solrParams.put(CommonParams.Q, "*:*");
|
||||
//solrParams.put(CommonParams.QT, "/export");
|
||||
|
||||
|
@ -114,15 +115,15 @@ public class SolrTable extends AbstractQueryableTable implements TranslatableTab
|
|||
|
||||
return new AbstractEnumerable<Object>() {
|
||||
public Enumerator<Object> enumerator() {
|
||||
TupleStream cloudSolrStream;
|
||||
TupleStream tupleStream;
|
||||
try {
|
||||
cloudSolrStream = new CloudSolrStream(zk, collection, solrParams);
|
||||
cloudSolrStream.open();
|
||||
String zk = properties.getProperty("zk");
|
||||
tupleStream = new CloudSolrStream(zk, collection, solrParams);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return new SolrEnumerator(cloudSolrStream, fields);
|
||||
return new SolrEnumerator(tupleStream, fields);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -143,7 +144,7 @@ public class SolrTable extends AbstractQueryableTable implements TranslatableTab
|
|||
|
||||
public Enumerator<T> enumerator() {
|
||||
//noinspection unchecked
|
||||
final Enumerable<T> enumerable = (Enumerable<T>) getTable().query(getZK());
|
||||
final Enumerable<T> enumerable = (Enumerable<T>) getTable().query(getProperties());
|
||||
return enumerable.enumerator();
|
||||
}
|
||||
|
||||
|
@ -151,8 +152,8 @@ public class SolrTable extends AbstractQueryableTable implements TranslatableTab
|
|||
return (SolrTable) table;
|
||||
}
|
||||
|
||||
private String getZK() {
|
||||
return schema.unwrap(SolrSchema.class).zk;
|
||||
private Properties getProperties() {
|
||||
return schema.unwrap(SolrSchema.class).properties;
|
||||
}
|
||||
|
||||
/** Called via code-generation.
|
||||
|
@ -160,8 +161,9 @@ public class SolrTable extends AbstractQueryableTable implements TranslatableTab
|
|||
* @see SolrMethod#SOLR_QUERYABLE_QUERY
|
||||
*/
|
||||
@SuppressWarnings("UnusedDeclaration")
|
||||
public Enumerable<Object> query(List<String> fields, List<String> filterQueries, List<String> order, String limit) {
|
||||
return getTable().query(getZK(), fields, filterQueries, order, limit);
|
||||
public Enumerable<Object> query(List<String> fields, List<String> filterQueries,
|
||||
List<String> order, String limit) {
|
||||
return getTable().query(getProperties(), fields, filterQueries, order, limit);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
9bc0eef759c2e341bfa206cc3e21d685037ad05f
|
|
@ -0,0 +1 @@
|
|||
5471bd9acf303dacda789a150c3059a9f9a0f9fc
|
|
@ -0,0 +1 @@
|
|||
00686c73a7b41a1931bdda43952d88bed92dc827
|
|
@ -0,0 +1 @@
|
|||
b71e76d942b33dfa26e4e3047ff2a774d1f917b4
|
|
@ -0,0 +1 @@
|
|||
a941956b3a4664d0cf728ece06ba25cc2110a3aa
|
|
@ -0,0 +1 @@
|
|||
37fde5de7edd5d7ebe075f03f4c083df2ac73dd8
|
|
@ -1 +0,0 @@
|
|||
b4261cbabfd2f28b8d4d20d7e3a3d1be48bb890c
|
|
@ -1 +0,0 @@
|
|||
159a81631ed2cc1bc865f3d8e51239c9e8a20bea
|
|
@ -25,16 +25,12 @@ import java.sql.ResultSet;
|
|||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.Statement;
|
||||
import java.sql.Types;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.cloud.AbstractZkTestCase;
|
||||
import org.apache.solr.common.cloud.DocCollection;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -214,21 +210,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals("hello3", rs.getString("a_s"));
|
||||
assertEquals("hello3", rs.getString(1));
|
||||
assertEquals(26, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(26, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(26, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello0", rs.getString("a_s"));
|
||||
assertEquals("hello0", rs.getString(1));
|
||||
assertEquals(18, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(18, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(18, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello4", rs.getString("a_s"));
|
||||
assertEquals("hello4", rs.getString(1));
|
||||
assertEquals(11, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(11, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(11, rs.getDouble(2), 0);
|
||||
|
||||
assertFalse(rs.next());
|
||||
|
@ -249,21 +245,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals("hello3", rs.getString("a_s"));
|
||||
assertEquals("hello3", rs.getString(1));
|
||||
assertEquals(26, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(26, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(26, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello0", rs.getString("a_s"));
|
||||
assertEquals("hello0", rs.getString(1));
|
||||
assertEquals(18, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(18, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(18, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello4", rs.getString("a_s"));
|
||||
assertEquals("hello4", rs.getString(1));
|
||||
assertEquals(11, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(11, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(11, rs.getDouble(2), 0);
|
||||
|
||||
assertFalse(rs.next());
|
||||
|
@ -288,21 +284,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals("hello3", rs.getString("a_s"));
|
||||
assertEquals("hello3", rs.getString(1));
|
||||
assertEquals(26, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(26, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(26, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello0", rs.getString("a_s"));
|
||||
assertEquals("hello0", rs.getString(1));
|
||||
assertEquals(18, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(18, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(18, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello4", rs.getString("a_s"));
|
||||
assertEquals("hello4", rs.getString(1));
|
||||
assertEquals(11, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(11, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(11, rs.getDouble(2), 0);
|
||||
|
||||
assertFalse(rs.next());
|
||||
|
@ -328,21 +324,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals("hello3", rs.getString("a_s"));
|
||||
assertEquals("hello3", rs.getString(1));
|
||||
assertEquals(26, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(26, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(26, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello0", rs.getString("a_s"));
|
||||
assertEquals("hello0", rs.getString(1));
|
||||
assertEquals(18, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(18, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(18, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello4", rs.getString("a_s"));
|
||||
assertEquals("hello4", rs.getString(1));
|
||||
assertEquals(11, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(11, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(11, rs.getDouble(2), 0);
|
||||
|
||||
assertFalse(rs.next());
|
||||
|
@ -373,21 +369,21 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertEquals("hello3", rs.getString("a_s"));
|
||||
assertEquals("hello3", rs.getString(1));
|
||||
assertEquals(26, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(26, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(26, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello0", rs.getString("a_s"));
|
||||
assertEquals("hello0", rs.getString(1));
|
||||
assertEquals(18, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(18, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(18, rs.getDouble(2), 0);
|
||||
|
||||
assertTrue(rs.next());
|
||||
|
||||
assertEquals("hello4", rs.getString("a_s"));
|
||||
assertEquals("hello4", rs.getString(1));
|
||||
assertEquals(11, rs.getDouble("sum(a_f)"), 0);
|
||||
assertEquals(11, rs.getDouble("EXPR$1"), 0); //sum(a_f)
|
||||
assertEquals(11, rs.getDouble(2), 0);
|
||||
|
||||
assertFalse(rs.next());
|
||||
|
@ -395,7 +391,8 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
// TODO fix error checking
|
||||
/*
|
||||
//Test error propagation
|
||||
props = new Properties();
|
||||
props.put("aggregationMode", "facet");
|
||||
|
@ -405,10 +402,11 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
"order by sum(a_f) desc")) {
|
||||
} catch (Exception e) {
|
||||
String errorMessage = e.getMessage();
|
||||
assertTrue(errorMessage.contains("Group by queries must include atleast one aggregate function"));
|
||||
assertTrue(errorMessage.contains("Group by queries must include at least one aggregate function"));
|
||||
}
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
testDriverMetadata();
|
||||
}
|
||||
|
@ -468,6 +466,8 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
// assertEquals(0, databaseMetaData.getDriverMajorVersion());
|
||||
// assertEquals(0, databaseMetaData.getDriverMinorVersion());
|
||||
|
||||
// TODO fix getCatalogs, getSchemas, and getTables
|
||||
/*
|
||||
try(ResultSet rs = databaseMetaData.getCatalogs()) {
|
||||
assertTrue(rs.next());
|
||||
assertEquals(zkServer.getZkAddress(), rs.getString("TABLE_CAT"));
|
||||
|
@ -493,6 +493,7 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
}
|
||||
assertFalse(rs.next());
|
||||
}
|
||||
*/
|
||||
|
||||
assertTrue(con.isReadOnly());
|
||||
con.setReadOnly(true);
|
||||
|
@ -580,7 +581,7 @@ public class JdbcTest extends AbstractFullDistribZkTestBase {
|
|||
assertEquals("id", resultSetMetaData.getColumnName(1));
|
||||
assertEquals("a_i", resultSetMetaData.getColumnName(2));
|
||||
assertEquals("a_s", resultSetMetaData.getColumnName(3));
|
||||
assertEquals("a_f", resultSetMetaData.getColumnName(4));
|
||||
assertEquals("my_float_col", resultSetMetaData.getColumnName(4));
|
||||
assertEquals("testnull_i", resultSetMetaData.getColumnName(5));
|
||||
|
||||
assertEquals("id", resultSetMetaData.getColumnLabel(1));
|
||||
|
|
Loading…
Reference in New Issue