diff --git a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java index eeee61f4699..c29500bb695 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java @@ -85,7 +85,7 @@ public class TermsQuery extends Query implements Accountable { // Same threshold as MultiTermQueryConstantScoreWrapper static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16; - private final Set fields; + private final boolean singleField; // whether all terms are from the same field private final PrefixCodedTerms termData; private final int termDataHashCode; // cached hashcode of termData @@ -110,7 +110,7 @@ public class TermsQuery extends Query implements Accountable { } previous = term; } - this.fields = Collections.unmodifiableSet(fields); + singleField = fields.size() == 1; termData = builder.finish(); termDataHashCode = termData.hashCode(); } @@ -137,7 +137,7 @@ public class TermsQuery extends Query implements Accountable { builder.add(field, term); previous.copyBytes(term); } - fields = Collections.singleton(field); + singleField = true; termData = builder.finish(); termDataHashCode = termData.hashCode(); } @@ -307,7 +307,7 @@ public class TermsQuery extends Query implements Accountable { matchingTerms.add(new TermAndState(field, termsEnum)); } else { assert matchingTerms.size() == threshold; - if (TermsQuery.this.fields.size() == 1) { + if (singleField) { // common case: all terms are in the same field // use an optimized builder that leverages terms stats to be more efficient builder = new DocIdSetBuilder(reader.maxDoc(), terms); diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index c66586da09d..9c91a5ff0bf 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -177,6 +177,13 @@ Bug Fixes * SOLR-9059: NPE in SolrClientCache following collection reload (Joel Bernstein, Ryan Yacyshyn) +* SOLR-8792: ZooKeeper ACL support fixed. (Esther Quansah, Ishan Chattopadhyaya, Steve Rowe) + +* SOLR-9064: Adds an explanation of the incoming stream to an UpdateStream's explanation (Dennis Gove) + +* SOLR-9030: The 'downnode' overseer command can trip asserts in ZkStateWriter. + (Scott Blum, Mark Miller, shalin) + Optimizations ---------------------- * SOLR-8722: Don't force a full ZkStateReader refresh on every Overseer operation. @@ -239,7 +246,11 @@ Other Changes * SOLR-9047: zkcli should allow alternative locations for log4j configuration (Gregory Chanan) -* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, janhoy) +* SOLR-9053: Upgrade commons-fileupload to 1.3.1, fixing a potential vulnerability (Jeff Field, Mike Drob via janhoy) + +* SOLR-9066 Make CountMetric return long instead of double (Kevin Risden) + +* SOLR-9065: Migrate SolrJ distributed tests to SolrCloudTestCase. (Alan Woodward) ================== 6.0.0 ================== diff --git a/solr/bin/solr b/solr/bin/solr index 9cf8ae53d76..aaf2f3dadcf 100755 --- a/solr/bin/solr +++ b/solr/bin/solr @@ -424,7 +424,7 @@ function jetty_port() { # useful for doing cross-platform work from the command-line using Java function run_tool() { - "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS -Dsolr.install.dir="$SOLR_TIP" \ + "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS $SOLR_ZK_CREDS_AND_ACLS -Dsolr.install.dir="$SOLR_TIP" \ -Dlog4j.configuration="file:$DEFAULT_SERVER_DIR/scripts/cloud-scripts/log4j.properties" \ -classpath "$DEFAULT_SERVER_DIR/solr-webapp/webapp/WEB-INF/lib/*:$DEFAULT_SERVER_DIR/lib/ext/*" \ org.apache.solr.util.SolrCLI "$@" diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd index 6dce9b9f95c..a8d71c43bf2 100644 --- a/solr/bin/solr.cmd +++ b/solr/bin/solr.cmd @@ -918,7 +918,7 @@ IF "%FG%"=="1" ( echo %SOLR_PORT%>"%SOLR_TIP%"\bin\solr-%SOLR_PORT%.port REM now wait to see Solr come online ... - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI status -maxWaitSecs 30 -solr !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:%SOLR_PORT%/solr ) @@ -928,7 +928,7 @@ goto done :run_example REM Run the requested example -"%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ +"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI run_example -script "%SDIR%\solr.cmd" -e %EXAMPLE% -d "%SOLR_SERVER_DIR%" -urlScheme !SOLR_URL_SCHEME! !PASS_TO_RUN_EXAMPLE! @@ -948,7 +948,7 @@ for /f "usebackq" %%i in (`dir /b "%SOLR_TIP%\bin" ^| findstr /i "^solr-.*\.port @echo. set has_info=1 echo Found Solr process %%k running on port !SOME_SOLR_PORT! - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI status -solr !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:!SOME_SOLR_PORT!/solr @echo. @@ -987,13 +987,13 @@ goto parse_healthcheck_args :run_healthcheck IF NOT DEFINED HEALTHCHECK_COLLECTION goto healthcheck_usage IF NOT DEFINED HEALTHCHECK_ZK_HOST set "HEALTHCHECK_ZK_HOST=localhost:9983" -"%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ +"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI healthcheck -collection !HEALTHCHECK_COLLECTION! -zkHost !HEALTHCHECK_ZK_HOST! goto done :get_version -"%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ +"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI version goto done @@ -1082,12 +1082,12 @@ if "!CREATE_PORT!"=="" ( ) if "%SCRIPT_CMD%"=="create_core" ( - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI create_core -name !CREATE_NAME! -solrUrl !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:!CREATE_PORT!/solr ^ -confdir !CREATE_CONFDIR! -configsetsDir "%SOLR_TIP%\server\solr\configsets" ) else ( - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI create -name !CREATE_NAME! -shards !CREATE_NUM_SHARDS! -replicationFactor !CREATE_REPFACT! ^ -confname !CREATE_CONFNAME! -confdir !CREATE_CONFDIR! -configsetsDir "%SOLR_TIP%\server\solr\configsets" -solrUrl !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:!CREATE_PORT!/solr @@ -1153,7 +1153,7 @@ if "!DELETE_CONFIG!"=="" ( set DELETE_CONFIG=true ) -"%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ +"%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI delete -name !DELETE_NAME! -deleteConfig !DELETE_CONFIG! ^ -solrUrl !SOLR_URL_SCHEME!://%SOLR_TOOL_HOST%:!DELETE_PORT!/solr @@ -1225,11 +1225,11 @@ if "!CONFIGSET_ZK!"=="" ( ) IF "!ZK_OP!"=="upconfig" ( - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !CONFIGSET_ZK! -configsetsDir "%SOLR_TIP%/server/solr/configsets" ) ELSE ( - "%JAVA%" %SOLR_SSL_OPTS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ + "%JAVA%" %SOLR_SSL_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI !ZK_OP! -confname !CONFIGSET_NAME! -confdir !CONFIGSET_DIR! -zkHost !CONFIGSET_ZK! ) diff --git a/solr/bin/solr.in.cmd b/solr/bin/solr.in.cmd index 9a8231cc1b1..76c77ed7f1e 100644 --- a/solr/bin/solr.in.cmd +++ b/solr/bin/solr.in.cmd @@ -104,3 +104,10 @@ REM set SOLR_SSL_CLIENT_KEY_STORE= REM set SOLR_SSL_CLIENT_KEY_STORE_PASSWORD= REM set SOLR_SSL_CLIENT_TRUST_STORE= REM setSOLR_SSL_CLIENT_TRUST_STORE_PASSWORD= + +REM Settings for ZK ACL +REM set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider ^ +REM -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider ^ +REM -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^ +REM -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD +REM set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS% diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh index 150f6c277d3..e5f8503bc58 100644 --- a/solr/bin/solr.in.sh +++ b/solr/bin/solr.in.sh @@ -118,3 +118,10 @@ SOLR_OPTS="$SOLR_OPTS -Xss256k" #SOLR_AUTHENTICATION_CLIENT_CONFIGURER= #SOLR_AUTHENTICATION_OPTS= +# Settings for ZK ACL +#SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider \ +# -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider \ +# -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \ +# -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD" +#SOLR_OPTS="$SOLR_OPTS $SOLR_ZK_CREDS_AND_ACLS" + diff --git a/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java b/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java index f7779510531..60f383ce6ee 100644 --- a/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java +++ b/solr/core/src/java/org/apache/solr/client/solrj/embedded/SSLConfig.java @@ -77,38 +77,63 @@ public class SSLConfig { /** * Returns an SslContextFactory that should be used by a jetty server based on the specified - * configuration, or null if no SSL should be used. + * SSLConfig param which may be null. * - * The specified sslConfig will be completely ignored if the "tests.jettySsl" system property is - * true - in which case standard "javax.net.ssl.*" system properties will be used instead, along - * with "tests.jettySsl.clientAuth" + * if the SSLConfig param is non-null, then this method will return the results of + * {@link #createContextFactory()}. * - * @see #isSSLMode + * If the SSLConfig param is null, then this method will return null unless the + * tests.jettySsl system property is true, in which case standard "javax.net.ssl.*" + * system properties will be used instead, along with "tests.jettySsl.clientAuth". + * + * @see #createContextFactory() */ public static SslContextFactory createContextFactory(SSLConfig sslConfig) { - if (sslConfig == null) { - if (Boolean.getBoolean("tests.jettySsl")) { - return configureSslFromSysProps(); - } + if (sslConfig != null) { + return sslConfig.createContextFactory(); + } + // else... + if (Boolean.getBoolean("tests.jettySsl")) { + return configureSslFromSysProps(); + } + // else... + return null; + } + + /** + * Returns an SslContextFactory that should be used by a jetty server based on this SSLConfig instance, + * or null if SSL should not be used. + * + * The default implementation generates a simple factory according to the keystore, truststore, + * and clientAuth properties of this object. + * + * @see #getKeyStore + * @see #getKeyStorePassword + * @see #isClientAuthMode + * @see #getTrustStore + * @see #getTrustStorePassword + */ + public SslContextFactory createContextFactory() { + + if (! isSSLMode()) { return null; } - - if (!sslConfig.isSSLMode()) - return null; - - SslContextFactory factory = new SslContextFactory(false); - if (sslConfig.getKeyStore() != null) - factory.setKeyStorePath(sslConfig.getKeyStore()); - if (sslConfig.getKeyStorePassword() != null) - factory.setKeyStorePassword(sslConfig.getKeyStorePassword()); - factory.setNeedClientAuth(sslConfig.isClientAuthMode()); + // else... - if (sslConfig.isClientAuthMode()) { - if (sslConfig.getTrustStore() != null) - factory.setTrustStorePath(sslConfig.getTrustStore()); - if (sslConfig.getTrustStorePassword() != null) - factory.setTrustStorePassword(sslConfig.getTrustStorePassword()); + SslContextFactory factory = new SslContextFactory(false); + if (getKeyStore() != null) + factory.setKeyStorePath(getKeyStore()); + if (getKeyStorePassword() != null) + factory.setKeyStorePassword(getKeyStorePassword()); + + factory.setNeedClientAuth(isClientAuthMode()); + + if (isClientAuthMode()) { + if (getTrustStore() != null) + factory.setTrustStorePath(getTrustStore()); + if (getTrustStorePassword() != null) + factory.setTrustStorePassword(getTrustStorePassword()); } return factory; diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java index 92310ed5f05..d313a47043a 100644 --- a/solr/core/src/java/org/apache/solr/cloud/Assign.java +++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java @@ -233,8 +233,9 @@ public class Assign { } DocCollection coll = clusterState.getCollection(collectionName); Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1); - for (String s : clusterState.getCollections()) { - DocCollection c = clusterState.getCollection(s); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + DocCollection c = entry.getValue(); //identify suitable nodes by checking the no:of cores in each of them for (Slice slice : c.getSlices()) { Collection replicas = slice.getReplicas(); @@ -242,7 +243,7 @@ public class Assign { ReplicaCount count = nodeNameVsShardCount.get(replica.getNodeName()); if (count != null) { count.totalNodes++; // Used ot "weigh" whether this node should be used later. - if (s.equals(collectionName)) { + if (entry.getKey().equals(collectionName)) { count.thisCollectionNodes++; if (count.thisCollectionNodes >= maxShardsPerNode) nodeNameVsShardCount.remove(replica.getNodeName()); } diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java index e8ac6c5ddf6..bbb45aacafd 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java @@ -162,10 +162,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { liveNodes = clusterState.getLiveNodes(); lastClusterStateVersion = clusterState.getZkClusterStateVersion(); - Set collections = clusterState.getCollections(); - for (final String collection : collections) { - log.debug("look at collection={}", collection); - DocCollection docCollection = clusterState.getCollection(collection); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + log.debug("look at collection={}", entry.getKey()); + DocCollection docCollection = entry.getValue(); if (!docCollection.getAutoAddReplicas()) { log.debug("Collection {} is not setup to use autoAddReplicas, skipping..", docCollection.getName()); continue; @@ -174,7 +174,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { log.debug("Skipping collection because it has no defined replicationFactor, name={}", docCollection.getName()); continue; } - log.debug("Found collection, name={} replicationFactor={}", collection, docCollection.getReplicationFactor()); + log.debug("Found collection, name={} replicationFactor={}", entry.getKey(), docCollection.getReplicationFactor()); Collection slices = docCollection.getSlices(); for (Slice slice : slices) { @@ -188,7 +188,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { if (downReplicas.size() > 0 && goodReplicas < docCollection.getReplicationFactor()) { // badReplicaMap.put(collection, badReplicas); - processBadReplicas(collection, downReplicas); + processBadReplicas(entry.getKey(), downReplicas); } else if (goodReplicas > docCollection.getReplicationFactor()) { log.debug("There are too many replicas"); } @@ -313,10 +313,11 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable { ClusterState clusterState = zkStateReader.getClusterState(); if (clusterState != null) { - Set collections = clusterState.getCollections(); - for (String collection : collections) { - log.debug("look at collection {} as possible create candidate", collection); - DocCollection docCollection = clusterState.getCollection(collection); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + String collection = entry.getKey(); + log.debug("look at collection {} as possible create candidate", collection); + DocCollection docCollection = entry.getValue(); // TODO - only operate on collections with sharedfs failover = true ?? Collection slices = docCollection.getSlices(); for (Slice slice : slices) { diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java index 15fed42182f..ba8f129811a 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java @@ -350,10 +350,10 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler { throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet does not exist to delete: " + configSetName); } - for (String s : zkStateReader.getClusterState().getCollections()) { - if (configSetName.equals(zkStateReader.readConfigName(s))) + for (Map.Entry entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) { + if (configSetName.equals(zkStateReader.readConfigName(entry.getKey()))) throw new SolrException(ErrorCode.BAD_REQUEST, - "Can not delete ConfigSet as it is currently being used by collection [" + s + "]"); + "Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]"); } String propertyPath = ConfigSetProperties.DEFAULT_FILENAME; diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index 3afc3b99892..ae73633cf33 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -689,14 +689,12 @@ public final class ZkController { long now = System.nanoTime(); long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS); boolean foundStates = true; - ClusterState clusterState = zkStateReader.getClusterState(); - Set collections = clusterState.getCollections(); - + while (System.nanoTime() < timeout) { - clusterState = zkStateReader.getClusterState(); - collections = clusterState.getCollections(); - for (String collectionName : collections) { - DocCollection collection = clusterState.getCollection(collectionName); + ClusterState clusterState = zkStateReader.getClusterState(); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + DocCollection collection = entry.getValue(); Collection slices = collection.getSlices(); for (Slice slice : slices) { Collection replicas = slice.getReplicas(); diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java index 5dd27c1735d..6ca07455a55 100644 --- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java +++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java @@ -49,14 +49,13 @@ public class NodeMutator { log.info("DownNode state invoked for node: " + nodeName); - Set collections = clusterState.getCollections(); - for (String collection : collections) { - DocCollection docCollection = clusterState.getCollection(collection); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + DocCollection docCollection = entry.getValue(); Map slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap()); - Set> entries = slicesCopy.entrySet(); - for (Entry entry : entries) { - Slice slice = docCollection.getSlice(entry.getKey()); + for (Entry sliceEntry : slicesCopy.entrySet()) { + Slice slice = docCollection.getSlice(sliceEntry.getKey()); Map newReplicas = new HashMap(); Collection replicas = slice.getReplicas(); @@ -77,7 +76,7 @@ public class NodeMutator { } - zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy))); + zkWriteCommands.add(new ZkWriteCommand(entry.getKey(), docCollection.copyWithSlices(slicesCopy))); } return zkWriteCommands; diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java index e9edef1b4eb..02ed873ef5b 100644 --- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java +++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java @@ -229,7 +229,6 @@ public class ZkStateWriter { byte[] data = Utils.toJSON(singletonMap(c.getName(), c)); if (reader.getZkClient().exists(path, true)) { log.info("going to update_collection {} version: {}", path, c.getZNodeVersion()); - assert c.getZNodeVersion() >= 0; Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true); DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path); clusterState = clusterState.copyWith(name, newCollection); @@ -251,13 +250,9 @@ public class ZkStateWriter { assert clusterState.getZkClusterStateVersion() >= 0; byte[] data = Utils.toJSON(clusterState); Stat stat = reader.getZkClient().setData(ZkStateReader.CLUSTER_STATE, data, clusterState.getZkClusterStateVersion(), true); - Set collectionNames = clusterState.getCollections(); - Map collectionStates = new HashMap<>(collectionNames.size()); - for (String c : collectionNames) { - collectionStates.put(c, clusterState.getCollection(c)); - } + Map collections = clusterState.getCollectionsMap(); // use the reader's live nodes because our cluster state's live nodes may be stale - clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collectionStates); + clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collections); isClusterStateModified = false; } lastUpdatedTime = System.nanoTime(); diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java index 2d68a3c4671..efd06481aed 100644 --- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java +++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java @@ -104,8 +104,9 @@ public class ReplicaAssigner { validateTags(nodeVsTags); if (clusterState != null) { - for (String s : clusterState.getCollections()) { - DocCollection coll = clusterState.getCollection(s); + Map collections = clusterState.getCollectionsMap(); + for (Map.Entry entry : collections.entrySet()) { + DocCollection coll = entry.getValue(); for (Slice slice : coll.getSlices()) { for (Replica replica : slice.getReplicas()) { AtomicInteger count = nodeVsCores.get(replica.getNodeName()); diff --git a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java index c26c5a8db5e..b8df7f2dc90 100644 --- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java @@ -1515,8 +1515,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe CloudSolrClient cloudSolrClient = this.context.getSolrClientCache().getCloudSolrClient(this.zkHost); cloudSolrClient.connect(); ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader(); - if (zkStateReader.getClusterState().getCollections().size() != 0) { - this.tables.addAll(zkStateReader.getClusterState().getCollections()); + Map collections = zkStateReader.getClusterState().getCollectionsMap(); + if (collections.size() != 0) { + this.tables.addAll(collections.keySet()); } Collections.sort(this.tables); } diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java index 63044fa338f..48302cd818d 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java @@ -89,20 +89,22 @@ public class ClusterStatus { byte[] bytes = Utils.toJSON(clusterState); Map stateMap = (Map) Utils.fromJSON(bytes); - Set collections; String routeKey = message.getStr(ShardParams._ROUTE_); String shard = message.getStr(ZkStateReader.SHARD_ID_PROP); + + Map collectionsMap = null; if (collection == null) { - collections = new HashSet<>(clusterState.getCollections()); + collectionsMap = clusterState.getCollectionsMap(); } else { - collections = Collections.singleton(collection); + collectionsMap = Collections.singletonMap(collection, clusterState.getCollectionOrNull(collection)); } NamedList collectionProps = new SimpleOrderedMap<>(); - for (String name : collections) { + for (Map.Entry entry : collectionsMap.entrySet()) { Map collectionStatus; - DocCollection clusterStateCollection = clusterState.getCollectionOrNull(name); + String name = entry.getKey(); + DocCollection clusterStateCollection = entry.getValue(); if (clusterStateCollection == null) { if (collection != null) { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found"); diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java index 6652584f240..4deddedbf5d 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java @@ -679,11 +679,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission @Override Map call(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler handler) throws Exception { NamedList results = new NamedList<>(); - Set collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollections(); - List collectionList = new ArrayList<>(); - for (String collection : collections) { - collectionList.add(collection); - } + Map collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollectionsMap(); + List collectionList = new ArrayList<>(collections.keySet()); results.add("collections", collectionList); SolrResponse response = new OverseerSolrResponse(results); rsp.getValues().addAll(response.getResponse()); diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java index 6ba571af141..57a4d7b2eaf 100644 --- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java +++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java @@ -815,15 +815,15 @@ public class HttpSolrCall { private void getSlicesForCollections(ClusterState clusterState, Collection slices, boolean activeSlices) { if (activeSlices) { - for (String collection : clusterState.getCollections()) { - final Collection activeCollectionSlices = clusterState.getActiveSlices(collection); + for (Map.Entry entry : clusterState.getCollectionsMap().entrySet()) { + final Collection activeCollectionSlices = entry.getValue().getActiveSlices(); if (activeCollectionSlices != null) { slices.addAll(activeCollectionSlices); } } } else { - for (String collection : clusterState.getCollections()) { - final Collection collectionSlices = clusterState.getSlices(collection); + for (Map.Entry entry : clusterState.getCollectionsMap().entrySet()) { + final Collection collectionSlices = entry.getValue().getSlices(); if (collectionSlices != null) { slices.addAll(collectionSlices); } diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java index a91f6a26394..9d7e7d9aced 100644 --- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java +++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java @@ -57,7 +57,6 @@ import org.apache.solr.core.SolrConfig; import org.apache.solr.core.SolrCore; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequestBase; -import org.apache.solr.util.RTimer; import org.apache.solr.util.RTimerTree; import static org.apache.solr.common.params.CommonParams.PATH; @@ -564,21 +563,18 @@ public class SolrRequestParsers upload.setSizeMax( ((long) uploadLimitKB) * 1024L ); // Parse the request - List items = upload.parseRequest(req); - Iterator iter = items.iterator(); - while (iter.hasNext()) { - FileItem item = (FileItem) iter.next(); - - // If it's a form field, put it in our parameter map - if (item.isFormField()) { - MultiMapSolrParams.addParam( - item.getFieldName().trim(), - item.getString(), params.getMap() ); - } - // Add the stream - else { - streams.add( new FileItemContentStream( item ) ); - } + List items = upload.parseRequest(req); + for (FileItem item : items) { + // If it's a form field, put it in our parameter map + if (item.isFormField()) { + MultiMapSolrParams.addParam( + item.getFieldName().trim(), + item.getString(), params.getMap() ); + } + // Add the stream + else { + streams.add( new FileItemContentStream( item ) ); + } } return params; } diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java index d21400d61e1..b5960f39dc5 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -1883,7 +1883,7 @@ public class SolrCLI { log.warn("Skipping safety checks, configuration directory "+configName+" will be deleted with impunity."); } else { // need to scan all Collections to see if any are using the config - Set collections = zkStateReader.getClusterState().getCollections(); + Set collections = zkStateReader.getClusterState().getCollectionsMap().keySet(); // give a little note to the user if there are many collections in case it takes a while if (collections.size() > 50) diff --git a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java index a25498aef25..e88afe9d28f 100644 --- a/solr/core/src/test/org/apache/solr/TestDistributedSearch.java +++ b/solr/core/src/test/org/apache/solr/TestDistributedSearch.java @@ -28,6 +28,7 @@ import java.util.Map; import org.apache.commons.lang.StringUtils; import org.apache.lucene.util.LuceneTestCase.Slow; +import org.apache.solr.SolrTestCaseJ4.SuppressSSL; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrServerException; @@ -69,6 +70,7 @@ import org.slf4j.LoggerFactory; * @since solr 1.3 */ @Slow +@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9061") public class TestDistributedSearch extends BaseDistributedSearchTestCase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java index d0005afd58d..875a7f8b7bb 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateTest.java @@ -66,21 +66,21 @@ public class ClusterStateTest extends SolrTestCaseJ4 { assertEquals("Provided liveNodes not used properly", 2, loadedClusterState .getLiveNodes().size()); - assertEquals("No collections found", 2, loadedClusterState.getCollections().size()); - assertEquals("Poperties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1")); - assertEquals("Poperties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2")); + assertEquals("No collections found", 2, loadedClusterState.getCollectionsMap().size()); + assertEquals("Properties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1")); + assertEquals("Properties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2")); loadedClusterState = ClusterState.load(-1, new byte[0], liveNodes); assertEquals("Provided liveNodes not used properly", 2, loadedClusterState .getLiveNodes().size()); - assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size()); + assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size()); loadedClusterState = ClusterState.load(-1, (byte[])null, liveNodes); assertEquals("Provided liveNodes not used properly", 2, loadedClusterState .getLiveNodes().size()); - assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size()); + assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size()); } public static ZkStateReader getMockZkStateReader(final Set collections) { diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java index 7dd77d96a75..6ada6bb5d98 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java @@ -1088,7 +1088,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa } } else { throw new IllegalArgumentException("Could not find collection in :" - + clusterState.getCollections()); + + clusterState.getCollectionsMap()); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java index 7863899504c..6562398560c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java @@ -525,7 +525,7 @@ public class OverseerTest extends SolrTestCaseJ4 { int maxIterations = 100; while (0 < maxIterations--) { final ClusterState state = stateReader.getClusterState(); - Set availableCollections = state.getCollections(); + Set availableCollections = state.getCollectionsMap().keySet(); int availableCount = 0; for(String requiredCollection: collections) { if(availableCollections.contains(requiredCollection)) { @@ -911,8 +911,9 @@ public class OverseerTest extends SolrTestCaseJ4 { ClusterState state = reader.getClusterState(); int numFound = 0; - for (String c : state.getCollections()) { - DocCollection collection = state.getCollection(c); + Map collectionsMap = state.getCollectionsMap(); + for (Map.Entry entry : collectionsMap.entrySet()) { + DocCollection collection = entry.getValue(); for (Slice slice : collection.getSlices()) { if (slice.getReplicasMap().get("core_node1") != null) { numFound++; diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java index 0af9e648f4f..2dba20b552e 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudBackupRestore.java @@ -87,7 +87,9 @@ public class TestCloudBackupRestore extends SolrCloudTestCase { } } - create.process(cluster.getSolrClient()); + CloudSolrClient solrClient = cluster.getSolrClient(); + create.process(solrClient); + indexDocs(collectionName); if (!isImplicit && random().nextBoolean()) { @@ -95,14 +97,14 @@ public class TestCloudBackupRestore extends SolrCloudTestCase { int prevActiveSliceCount = getActiveSliceCount(collectionName); CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(collectionName); splitShard.setShardName("shard1"); - splitShard.process(cluster.getSolrClient()); + splitShard.process(solrClient); // wait until we see one more active slice... for (int i = 0; getActiveSliceCount(collectionName) != prevActiveSliceCount + 1; i++) { assertTrue(i < 30); Thread.sleep(500); } // issue a hard commit. Split shard does a soft commit which isn't good enough for the backup/snapshooter to see - cluster.getSolrClient().commit(); + solrClient.commit(collectionName); } testBackupAndRestore(collectionName); @@ -120,7 +122,6 @@ public class TestCloudBackupRestore extends SolrCloudTestCase { return; } CloudSolrClient client = cluster.getSolrClient(); - client.setDefaultCollection(collectionName); List docs = new ArrayList<>(numDocs); for (int i=0; i collections = new ArrayList<>(); - collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollections()); + collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollectionsMap().keySet()); Collections.sort(collections); for (Tuple tuple : tuples) { assertEquals(zkServer.getZkAddress(), tuple.getString("TABLE_CAT")); @@ -2510,7 +2510,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assertEquals(2, tuples.size()); collections = new ArrayList<>(); - collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollections()); + collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollectionsMap().keySet()); Collections.sort(collections); for (Tuple tuple : tuples) { assertEquals(zkServer.getZkAddress(), tuple.getString("TABLE_CAT")); diff --git a/solr/core/src/test/org/apache/solr/handler/component/TestDistributedStatsComponentCardinality.java b/solr/core/src/test/org/apache/solr/handler/component/TestDistributedStatsComponentCardinality.java index 0ba9978e7f8..c7036836a2c 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/TestDistributedStatsComponentCardinality.java +++ b/solr/core/src/test/org/apache/solr/handler/component/TestDistributedStatsComponentCardinality.java @@ -27,6 +27,7 @@ import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.solr.BaseDistributedSearchTestCase; +import org.apache.solr.SolrTestCaseJ4.SuppressSSL; import org.apache.solr.client.solrj.response.FieldStatsInfo; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.params.SolrParams; @@ -40,6 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Slow +@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9062") public class TestDistributedStatsComponentCardinality extends BaseDistributedSearchTestCase { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); diff --git a/solr/server/scripts/cloud-scripts/zkcli.bat b/solr/server/scripts/cloud-scripts/zkcli.bat index 2e7e75deaa5..0e4359ceac1 100644 --- a/solr/server/scripts/cloud-scripts/zkcli.bat +++ b/solr/server/scripts/cloud-scripts/zkcli.bat @@ -15,4 +15,11 @@ if defined LOG4J_PROPS ( set "LOG4J_CONFIG=file:%SDIR%\log4j.properties" ) -"%JVM%" -Dlog4j.configuration="%LOG4J_CONFIG%" -classpath "%SDIR%\..\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\..\lib\ext\*" org.apache.solr.cloud.ZkCLI %* +REM Settings for ZK ACL +REM set SOLR_ZK_CREDS_AND_ACLS=-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider ^ +REM -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider ^ +REM -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^ +REM -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD + +"%JVM%" %SOLR_ZK_CREDS_AND_ACLS% -Dlog4j.configuration="%LOG4J_CONFIG%" ^ +-classpath "%SDIR%\..\..\solr-webapp\webapp\WEB-INF\lib\*;%SDIR%\..\..\lib\ext\*" org.apache.solr.cloud.ZkCLI %* diff --git a/solr/server/scripts/cloud-scripts/zkcli.sh b/solr/server/scripts/cloud-scripts/zkcli.sh index d049be0ff46..e37b6dacfc3 100755 --- a/solr/server/scripts/cloud-scripts/zkcli.sh +++ b/solr/server/scripts/cloud-scripts/zkcli.sh @@ -15,5 +15,12 @@ else log4j_config="file:$sdir/log4j.properties" fi -PATH=$JAVA_HOME/bin:$PATH $JVM -Dlog4j.configuration=$log4j_config -classpath "$sdir/../../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../../lib/ext/*" org.apache.solr.cloud.ZkCLI ${1+"$@"} +# Settings for ZK ACL +#SOLR_ZK_CREDS_AND_ACLS="-DzkACLProvider=org.apache.solr.common.cloud.VMParamsAllAndReadonlyDigestZkACLProvider \ +# -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCredentialsDigestZkCredentialsProvider \ +# -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD \ +# -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD" + +PATH=$JAVA_HOME/bin:$PATH $JVM $SOLR_ZK_CREDS_AND_ACLS -Dlog4j.configuration=$log4j_config \ +-classpath "$sdir/../../solr-webapp/webapp/WEB-INF/lib/*:$sdir/../../lib/ext/*" org.apache.solr.cloud.ZkCLI ${1+"$@"} diff --git a/solr/server/solr/solr.xml b/solr/server/solr/solr.xml index e247452da4d..68b15ba0116 100644 --- a/solr/server/solr/solr.xml +++ b/solr/server/solr/solr.xml @@ -39,6 +39,8 @@ ${zkClientTimeout:30000} ${distribUpdateSoTimeout:600000} ${distribUpdateConnTimeout:60000} + ${zkCredentialsProvider:org.apache.solr.common.cloud.DefaultZkCredentialsProvider} + ${zkACLProvider:org.apache.solr.common.cloud.DefaultZkACLProvider} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java index b619f2facfd..b1659c91b1f 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java @@ -51,6 +51,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.ZkCoreNodeProps; @@ -329,16 +330,17 @@ public class CloudSolrStream extends TupleStream implements Expressible { Collection slices = clusterState.getActiveSlices(this.collection); - if(slices == null) { + if (slices == null) { //Try case insensitive match - for(String col : clusterState.getCollections()) { - if(col.equalsIgnoreCase(collection)) { - slices = clusterState.getActiveSlices(col); + Map collectionsMap = clusterState.getCollectionsMap(); + for (Map.Entry entry : collectionsMap.entrySet()) { + if (entry.getKey().equalsIgnoreCase(collection)) { + slices = entry.getValue().getActiveSlices(); break; } } - if(slices == null) { + if (slices == null) { throw new Exception("Collection not found:" + this.collection); } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java index 007c6442240..f4aabec8cf1 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java @@ -215,7 +215,7 @@ public class SolrStream extends TupleStream { throw new IOException("--> "+this.baseUrl+":"+e.getMessage()); } catch (Exception e) { //The Stream source did not provide an exception in a format that the SolrStream could propagate. - throw new IOException("--> "+this.baseUrl+": An exception has occurred on the server, refer to server log for details."); + throw new IOException("--> "+this.baseUrl+": An exception has occurred on the server, refer to server log for details.", e); } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java index 97a804d176d..ff441093187 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java @@ -463,16 +463,17 @@ public class TopicStream extends CloudSolrStream implements Expressible { Collection slices = clusterState.getActiveSlices(this.collection); - if(slices == null) { + if (slices == null) { //Try case insensitive match - for(String col : clusterState.getCollections()) { - if(col.equalsIgnoreCase(collection)) { - slices = clusterState.getActiveSlices(col); + Map collectionsMap = clusterState.getCollectionsMap(); + for (Map.Entry entry : collectionsMap.entrySet()) { + if (entry.getKey().equalsIgnoreCase(collection)) { + slices = entry.getValue().getActiveSlices(); break; } } - if(slices == null) { + if (slices == null) { throw new Exception("Collection not found:" + this.collection); } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java index 023ff56e2f4..5b1aae76b90 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java @@ -188,6 +188,7 @@ public class UpdateStream extends TupleStream implements Expressible { child.setImplementingClass(getClass().getName()); child.setExpressionType(ExpressionType.STREAM_DECORATOR); child.setExpression(toExpression(factory, false).toString()); + child.addChild(tupleSource.toExplanation(factory)); explanation.addChild(child); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Bucket.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Bucket.java index 8e2293c1801..54907ac0da8 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Bucket.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Bucket.java @@ -16,20 +16,14 @@ */ package org.apache.solr.client.solrj.io.stream.metrics; -import java.io.Serializable; import org.apache.solr.client.solrj.io.Tuple; -public class Bucket implements Serializable { +public class Bucket { - private static final long serialVersionUID = 1; private static final String NULL_VALUE = "NULL"; private String bucketKey; - public Bucket() { - - } - public Bucket(String bucketKey) { this.bucketKey = bucketKey; } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/CountMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/CountMetric.java index 445b530163d..0e8cbb057b8 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/CountMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/CountMetric.java @@ -16,7 +16,6 @@ */ package org.apache.solr.client.solrj.io.stream.metrics; import java.io.IOException; -import java.io.Serializable; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; @@ -24,15 +23,13 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -public class CountMetric extends Metric implements Serializable { - - private static final long serialVersionUID = 1; - +public class CountMetric extends Metric { private long count; public CountMetric(){ init("count"); } + public CountMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); @@ -63,7 +60,7 @@ public class CountMetric extends Metric implements Serializable { ++count; } - public double getValue() { + public Long getValue() { return count; } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MaxMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MaxMetric.java index 0594bf42249..c02b327d53c 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MaxMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MaxMetric.java @@ -17,7 +17,6 @@ package org.apache.solr.client.solrj.io.stream.metrics; import java.io.IOException; -import java.io.Serializable; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; @@ -25,11 +24,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -public class MaxMetric extends Metric implements Serializable { - - private static final long serialVersionUID = 1; - - public static final String MAX = "max"; +public class MaxMetric extends Metric { private long longMax = -Long.MIN_VALUE; private double doubleMax = -Double.MAX_VALUE; private String columnName; @@ -37,6 +32,7 @@ public class MaxMetric extends Metric implements Serializable { public MaxMetric(String columnName){ init("max", columnName); } + public MaxMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); @@ -58,8 +54,8 @@ public class MaxMetric extends Metric implements Serializable { setFunctionName(functionName); setIdentifier(functionName, "(", columnName, ")"); } - - public double getValue() { + + public Number getValue() { if(longMax == Long.MIN_VALUE) { return doubleMax; } else { @@ -68,8 +64,7 @@ public class MaxMetric extends Metric implements Serializable { } public String[] getColumns() { - String[] cols = {columnName}; - return cols; + return new String[]{columnName}; } public void update(Tuple tuple) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java index 097e04b822b..03c037a73ee 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java @@ -17,7 +17,6 @@ package org.apache.solr.client.solrj.io.stream.metrics; import java.io.IOException; -import java.io.Serializable; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; @@ -25,14 +24,12 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -public class MeanMetric extends Metric implements Serializable { +public class MeanMetric extends Metric { // How'd the MeanMetric get to be so mean? // Maybe it was born with it. // Maybe it was mayba-mean. // // I'll see myself out. - - private static final long serialVersionUID = 1; private String columnName; private double doubleSum; @@ -42,6 +39,7 @@ public class MeanMetric extends Metric implements Serializable { public MeanMetric(String columnName){ init("avg", columnName); } + public MeanMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); @@ -69,10 +67,10 @@ public class MeanMetric extends Metric implements Serializable { Object o = tuple.get(columnName); if(o instanceof Double) { Double d = (Double)tuple.get(columnName); - doubleSum += d.doubleValue(); + doubleSum += d; } else { Long l = (Long)tuple.get(columnName); - longSum += l.doubleValue(); + longSum += l; } } @@ -81,19 +79,16 @@ public class MeanMetric extends Metric implements Serializable { } public String[] getColumns() { - String[] cols = {columnName}; - return cols; + return new String[]{columnName}; } - public double getValue() { + public Double getValue() { double dcount = (double)count; if(longSum == 0) { - double ave = doubleSum/dcount; - return ave; + return doubleSum/dcount; } else { - double ave = longSum/dcount; - return ave; + return longSum/dcount; } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java index 4e2485483a7..582b54ae441 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java @@ -17,7 +17,6 @@ package org.apache.solr.client.solrj.io.stream.metrics; import java.io.IOException; -import java.io.Serializable; import java.util.UUID; import org.apache.solr.client.solrj.io.Tuple; @@ -26,19 +25,16 @@ import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; import org.apache.solr.client.solrj.io.stream.expr.Expressible; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -public abstract class Metric implements Serializable, Expressible { - - private static final long serialVersionUID = 1L; +public abstract class Metric implements Expressible { + private UUID metricNodeId = UUID.randomUUID(); private String functionName; private String identifier; - -// @Override + public String getFunctionName(){ return functionName; } - -// @Override + public void setFunctionName(String functionName){ this.functionName = functionName; } @@ -59,7 +55,7 @@ public abstract class Metric implements Serializable, Expressible { @Override public Explanation toExplanation(StreamFactory factory) throws IOException { - return new Explanation(metricNodeId.toString()) + return new Explanation(getMetricNodeId().toString()) .withFunctionName(functionName) .withImplementingClass(getClass().getName()) .withExpression(toExpression(factory).toString()) @@ -70,7 +66,7 @@ public abstract class Metric implements Serializable, Expressible { return metricNodeId; } - public abstract double getValue(); + public abstract Number getValue(); public abstract void update(Tuple tuple); public abstract Metric newInstance(); public abstract String[] getColumns(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MinMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MinMetric.java index 0a565809080..ddccde16e4b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MinMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MinMetric.java @@ -25,8 +25,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public class MinMetric extends Metric { - - private static final long serialVersionUID = 1L; private long longMin = Long.MAX_VALUE; private double doubleMin = Double.MAX_VALUE; @@ -35,6 +33,7 @@ public class MinMetric extends Metric { public MinMetric(String columnName){ init("min", columnName); } + public MinMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); @@ -57,13 +56,11 @@ public class MinMetric extends Metric { setIdentifier(functionName, "(", columnName, ")"); } - public String[] getColumns() { - String[] cols = {columnName}; - return cols; + return new String[]{columnName}; } - public double getValue() { + public Number getValue() { if(longMin == Long.MAX_VALUE) { return doubleMin; } else { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/SumMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/SumMetric.java index 578dae764fb..3eaf6d9e66f 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/SumMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/SumMetric.java @@ -17,7 +17,6 @@ package org.apache.solr.client.solrj.io.stream.metrics; import java.io.IOException; -import java.io.Serializable; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; @@ -25,10 +24,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -public class SumMetric extends Metric implements Serializable { - - private static final long serialVersionUID = 1; - +public class SumMetric extends Metric { private String columnName; private double doubleSum; private long longSum; @@ -36,6 +32,7 @@ public class SumMetric extends Metric implements Serializable { public SumMetric(String columnName){ init("sum", columnName); } + public SumMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); @@ -59,18 +56,17 @@ public class SumMetric extends Metric implements Serializable { } public String[] getColumns() { - String[] cols = {columnName}; - return cols; + return new String[]{columnName}; } public void update(Tuple tuple) { Object o = tuple.get(columnName); if(o instanceof Double) { Double d = (Double)o; - doubleSum += d.doubleValue(); + doubleSum += d; } else { Long l = (Long)o; - longSum += l.longValue(); + longSum += l; } } @@ -78,11 +74,11 @@ public class SumMetric extends Metric implements Serializable { return new SumMetric(columnName); } - public double getValue() { + public Number getValue() { if(longSum == 0) { return doubleSum; } else { - return (double)longSum; + return longSum; } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java index b7a9a60a845..452c7a15584 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java @@ -1042,6 +1042,10 @@ public abstract class CollectionAdminRequest return new RequestStatus(requestId); } + public static void waitForAsyncRequest(String requestId, SolrClient client, long timeout) throws SolrServerException, InterruptedException, IOException { + requestStatus(requestId).waitFor(client, timeout); + } + // REQUESTSTATUS request public static class RequestStatus extends CollectionAdminRequest { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java index d0f77599d51..f93a197783b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java @@ -30,13 +30,17 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Set; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.LBHttpSolrClient; +import org.apache.solr.client.solrj.response.UpdateResponse; import org.apache.solr.client.solrj.util.ClientUtils; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocRouter; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.util.ContentStream; import org.apache.solr.common.util.XML; @@ -103,6 +107,10 @@ public class UpdateRequest extends AbstractUpdateRequest { return this; } + public UpdateRequest add(String... fields) { + return add(new SolrInputDocument(fields)); + } + /** * Add a SolrInputDocument to this request * @param doc the document @@ -209,6 +217,13 @@ public class UpdateRequest extends AbstractUpdateRequest { deleteQuery.add(q); return this; } + + public UpdateResponse commit(SolrClient client, String collection) throws IOException, SolrServerException { + if (params == null) + params = new ModifiableSolrParams(); + params.set(UpdateParams.COMMIT, "true"); + return process(client, collection); + } /** * @param router to route updates with @@ -383,7 +398,7 @@ public class UpdateRequest extends AbstractUpdateRequest { /** * @since solr 1.4 */ - public void writeXML(Writer writer) throws IOException { + public UpdateRequest writeXML(Writer writer) throws IOException { List>> getDocLists = getDocLists(documents); for (Map> docs : getDocLists) { @@ -457,6 +472,7 @@ public class UpdateRequest extends AbstractUpdateRequest { } writer.append(""); } + return this; } // -------------------------------------------------------------------------- diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java index 6796300d6af..3d3c06014c0 100644 --- a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java +++ b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java @@ -38,8 +38,12 @@ public class SolrInputDocument extends SolrDocumentBase _childDocuments; - public SolrInputDocument() { + public SolrInputDocument(String... fields) { _fields = new LinkedHashMap<>(); + assert fields.length % 2 == 0; + for (int i = 0; i < fields.length; i += 2) { + addField(fields[i], fields[i + 1]); + } } public SolrInputDocument(Map fields) { diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java index 9cca0119d3f..55df27189bd 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java @@ -218,7 +218,10 @@ public class ClusterState implements JSONWriter.Writable { * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary * because the semantics of how collection list is loaded have changed in SOLR-6629. * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)} + * + * @deprecated use {@link #getCollectionsMap()} to avoid a second lookup for lazy collections */ + @Deprecated public Set getCollections() { Set result = new HashSet<>(); for (Entry entry : collectionStates.entrySet()) { @@ -229,6 +232,27 @@ public class ClusterState implements JSONWriter.Writable { return result; } + /** + * Get a map of collection name vs DocCollection objects + * + * Implementation note: This method resolves the collection reference by calling + * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary + * because the semantics of how collection list is loaded have changed in SOLR-6629. + * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)} + * + * @return a map of collection name vs DocCollection object + */ + public Map getCollectionsMap() { + Map result = new HashMap<>(collectionStates.size()); + for (Entry entry : collectionStates.entrySet()) { + DocCollection collection = entry.getValue().get(); + if (collection != null) { + result.put(entry.getKey(), collection); + } + } + return result; + } + /** * Get names of the currently live nodes. */ diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java index a298cb3ce8d..a5682451167 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java @@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -66,14 +67,14 @@ public class ClusterStateUtil { success = true; ClusterState clusterState = zkStateReader.getClusterState(); if (clusterState != null) { - Set collections; + Map collections = null; if (collection != null) { - collections = Collections.singleton(collection); + collections = Collections.singletonMap(collection, clusterState.getCollection(collection)); } else { - collections = clusterState.getCollections(); + collections = clusterState.getCollectionsMap(); } - for (String coll : collections) { - DocCollection docCollection = clusterState.getCollection(coll); + for (Map.Entry entry : collections.entrySet()) { + DocCollection docCollection = entry.getValue(); Collection slices = docCollection.getSlices(); for (Slice slice : slices) { // only look at active shards @@ -178,14 +179,14 @@ public class ClusterStateUtil { success = true; ClusterState clusterState = zkStateReader.getClusterState(); if (clusterState != null) { - Set collections; - if (collection == null) { - collections = clusterState.getCollections(); + Map collections = null; + if (collection != null) { + collections = Collections.singletonMap(collection, clusterState.getCollection(collection)); } else { - collections = Collections.singleton(collection); + collections = clusterState.getCollectionsMap(); } - for (String coll : collections) { - DocCollection docCollection = clusterState.getCollection(coll); + for (Map.Entry entry : collections.entrySet()) { + DocCollection docCollection = entry.getValue(); Collection slices = docCollection.getSlices(); for (Slice slice : slices) { // only look at active shards diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java index 9e4418c16f7..d30a73fa145 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java @@ -41,7 +41,8 @@ public class DocCollection extends ZkNodeProps { public static final String STATE_FORMAT = "stateFormat"; public static final String RULE = "rule"; public static final String SNITCH = "snitch"; - private int znodeVersion = -1; // sentinel + + private final int znodeVersion; private final String name; private final Map slices; @@ -55,7 +56,7 @@ public class DocCollection extends ZkNodeProps { public DocCollection(String name, Map slices, Map props, DocRouter router) { - this(name, slices, props, router, -1, ZkStateReader.CLUSTER_STATE); + this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE); } /** @@ -64,8 +65,9 @@ public class DocCollection extends ZkNodeProps { * @param props The properties of the slice. This is used directly and a copy is not made. */ public DocCollection(String name, Map slices, Map props, DocRouter router, int zkVersion, String znode) { - super(props==null ? props = new HashMap() : props); - this.znodeVersion = zkVersion; + super(props==null ? props = new HashMap<>() : props); + // -1 means any version in ZK CAS, so we choose Integer.MAX_VALUE instead to avoid accidental overwrites + this.znodeVersion = zkVersion == -1 ? Integer.MAX_VALUE : zkVersion; this.name = name; this.slices = slices; diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java index a11dbf26f11..55d8c83b386 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java @@ -520,7 +520,8 @@ public class ZkStateReader implements Closeable { * In fact this is a clever way to avoid doing a ZK exists check on * the /collections/collection_name/state.json znode * Such an exists check is done in {@link ClusterState#hasCollection(String)} and - * {@link ClusterState#getCollections()} method as a safeguard against exposing wrong collection names to the users + * {@link ClusterState#getCollections()} and {@link ClusterState#getCollectionsMap()} methods + * have a safeguard against exposing wrong collection names to the users */ private void refreshCollectionList(Watcher watcher) throws KeeperException, InterruptedException { synchronized (refreshCollectionListLock) { @@ -715,8 +716,7 @@ public class ZkStateReader implements Closeable { Map slices = clusterState.getSlicesMap(collection); if (slices == null) { throw new ZooKeeperException(ErrorCode.BAD_REQUEST, - "Could not find collection in zk: " + collection + " " - + clusterState.getCollections()); + "Could not find collection in zk: " + collection); } Slice replicas = slices.get(shardId); diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml similarity index 100% rename from solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-streaming.xml rename to solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/solrconfig.xml similarity index 100% rename from solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-streaming.xml rename to solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/solrconfig.xml diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java index 58643898256..616ddc47237 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java @@ -16,6 +16,19 @@ */ package org.apache.solr.client.solrj.impl; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeoutException; + import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -26,12 +39,13 @@ import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.request.AbstractUpdateRequest; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrDocumentList; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; @@ -46,6 +60,7 @@ import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -53,163 +68,87 @@ import org.junit.rules.ExpectedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.net.URL; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeoutException; - -import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES; -import static org.apache.solr.common.util.Utils.makeMap; -import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; -import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR; - /** * This test would be faster if we simulated the zk state instead. */ @Slow -public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); +public class CloudSolrClientTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); + private static final String COLLECTION = "collection1"; + + private static final String id = "id"; + + private static final int TIMEOUT = 30; @BeforeClass - public static void beforeSuperClass() { - // this is necessary because AbstractZkTestCase.buildZooKeeper is used by AbstractDistribZkTestBase - // and the auto-detected SOLRHOME=TEST_HOME() does not exist for solrj tests - // todo fix this - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME()); + public static void setupCluster() throws Exception { + configureCluster(3) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); + + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); } - protected String getCloudSolrConfig() { - return "solrconfig.xml"; - } - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - public static String SOLR_HOME() { - return SOLR_HOME; - } - - public CloudSolrClientTest() { - super(); - sliceCount = 2; - fixShardCount(3); + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + @Before + public void cleanIndex() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); } @Test - public void test() throws Exception { - testParallelUpdateQTime(); - checkCollectionParameters(); - allTests(); - stateVersionParamTest(); - customHttpClientTest(); - testOverwriteOption(); - preferLocalShardsTest(); - } - - private void testParallelUpdateQTime() throws Exception { + public void testParallelUpdateQTime() throws Exception { UpdateRequest req = new UpdateRequest(); for (int i=0; i<10; i++) { SolrInputDocument doc = new SolrInputDocument(); doc.addField("id", String.valueOf(TestUtil.nextInt(random(), 1000, 1100))); req.add(doc); } - UpdateResponse response = req.process(cloudClient); + UpdateResponse response = req.process(cluster.getSolrClient(), COLLECTION); // See SOLR-6547, we just need to ensure that no exception is thrown here assertTrue(response.getQTime() >= 0); } - private void testOverwriteOption() throws Exception, SolrServerException, - IOException { - String collectionName = "overwriteCollection"; - createCollection(collectionName, controlClientCloud, 1, 1); - waitForRecoveriesToFinish(collectionName, false); - try (CloudSolrClient cloudClient = createCloudClient(collectionName)) { - SolrInputDocument doc1 = new SolrInputDocument(); - doc1.addField(id, "0"); - doc1.addField("a_t", "hello1"); - SolrInputDocument doc2 = new SolrInputDocument(); - doc2.addField(id, "0"); - doc2.addField("a_t", "hello2"); - - UpdateRequest request = new UpdateRequest(); - request.add(doc1); - request.add(doc2); - request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false); - NamedList response = cloudClient.request(request); - QueryResponse resp = cloudClient.query(new SolrQuery("*:*")); - - assertEquals("There should be one document because overwrite=true", 1, resp.getResults().getNumFound()); - - doc1 = new SolrInputDocument(); - doc1.addField(id, "1"); - doc1.addField("a_t", "hello1"); - doc2 = new SolrInputDocument(); - doc2.addField(id, "1"); - doc2.addField("a_t", "hello2"); - - request = new UpdateRequest(); - // overwrite=false - request.add(doc1, false); - request.add(doc2, false); - request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false); - response = cloudClient.request(request); - - resp = cloudClient.query(new SolrQuery("*:*")); + @Test + public void testOverwriteOption() throws Exception { + + CollectionAdminRequest.createCollection("overwrite", "conf", 1, 1) + .processAndWait(cluster.getSolrClient(), TIMEOUT); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("overwrite", cluster.getSolrClient().getZkStateReader(), false, true, TIMEOUT); + + new UpdateRequest() + .add("id", "0", "a_t", "hello1") + .add("id", "0", "a_t", "hello2") + .commit(cluster.getSolrClient(), "overwrite"); + + QueryResponse resp = cluster.getSolrClient().query("overwrite", new SolrQuery("*:*")); + assertEquals("There should be one document because overwrite=true", 1, resp.getResults().getNumFound()); + + new UpdateRequest() + .add(new SolrInputDocument(id, "1", "a_t", "hello1"), /* overwrite = */ false) + .add(new SolrInputDocument(id, "1", "a_t", "hello2"), false) + .commit(cluster.getSolrClient(), "overwrite"); + + resp = cluster.getSolrClient().query("overwrite", new SolrQuery("*:*")); + assertEquals("There should be 3 documents because there should be two id=1 docs due to overwrite=false", 3, resp.getResults().getNumFound()); - assertEquals("There should be 3 documents because there should be two id=1 docs due to overwrite=false", 3, resp.getResults().getNumFound()); - } } - private void allTests() throws Exception { - - String collectionName = "clientTestExternColl"; - createCollection(collectionName, controlClientCloud, 2, 2); - waitForRecoveriesToFinish(collectionName, false); - CloudSolrClient cloudClient = createCloudClient(collectionName); - - assertNotNull(cloudClient); + @Test + public void testRouting() throws Exception { - handle.clear(); - handle.put("timestamp", SKIPVAL); - - waitForThingsToLevelOut(30); - - controlClient.deleteByQuery("*:*"); - cloudClient.deleteByQuery("*:*"); - - - controlClient.commit(); - this.cloudClient.commit(); - - SolrInputDocument doc1 = new SolrInputDocument(); - doc1.addField(id, "0"); - doc1.addField("a_t", "hello1"); - SolrInputDocument doc2 = new SolrInputDocument(); - doc2.addField(id, "2"); - doc2.addField("a_t", "hello2"); - - UpdateRequest request = new UpdateRequest(); - request.add(doc1); - request.add(doc2); - request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false); + AbstractUpdateRequest request = new UpdateRequest() + .add(id, "0", "a_t", "hello1") + .add(id, "2", "a_t", "hello2") + .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true); // Test single threaded routed updates for UpdateRequest - NamedList response = cloudClient.request(request); + NamedList response = cluster.getSolrClient().request(request, COLLECTION); CloudSolrClient.RouteResponse rr = (CloudSolrClient.RouteResponse) response; Map routes = rr.getRoutes(); Iterator> it = routes.entrySet() @@ -234,22 +173,19 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { // Test the deleteById routing for UpdateRequest - UpdateRequest delRequest = new UpdateRequest(); - delRequest.deleteById("0"); - delRequest.deleteById("2"); - delRequest.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false); - cloudClient.request(delRequest); - ModifiableSolrParams qParams = new ModifiableSolrParams(); - qParams.add("q", "*:*"); - QueryRequest qRequest = new QueryRequest(qParams); - QueryResponse qResponse = qRequest.process(cloudClient); + new UpdateRequest() + .deleteById("0") + .deleteById("2") + .commit(cluster.getSolrClient(), COLLECTION); + + QueryResponse qResponse = cluster.getSolrClient().query(COLLECTION, new SolrQuery("*:*")); SolrDocumentList docs = qResponse.getResults(); - assertTrue(docs.getNumFound() == 0); + assertEquals(0, docs.getNumFound()); // Test Multi-Threaded routed updates for UpdateRequest - try (CloudSolrClient threadedClient = getCloudSolrClient(zkServer.getZkAddress())) { + try (CloudSolrClient threadedClient = getCloudSolrClient(cluster.getZkServer().getZkAddress())) { threadedClient.setParallelUpdates(true); - threadedClient.setDefaultCollection(collectionName); + threadedClient.setDefaultCollection(COLLECTION); response = threadedClient.request(request); rr = (CloudSolrClient.RouteResponse) response; routes = rr.getRoutes(); @@ -277,13 +213,13 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { // Test that queries with _route_ params are routed by the client // Track request counts on each node before query calls - ClusterState clusterState = cloudClient.getZkStateReader().getClusterState(); - DocCollection col = clusterState.getCollection(collectionName); + ClusterState clusterState = cluster.getSolrClient().getZkStateReader().getClusterState(); + DocCollection col = clusterState.getCollection(COLLECTION); Map requestCountsMap = Maps.newHashMap(); for (Slice slice : col.getSlices()) { for (Replica replica : slice.getReplicas()) { String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP); - requestCountsMap.put(baseURL, getNumRequests(baseURL,collectionName)); + requestCountsMap.put(baseURL, getNumRequests(baseURL, COLLECTION)); } } @@ -328,7 +264,7 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { ModifiableSolrParams solrParams = new ModifiableSolrParams(); solrParams.set(CommonParams.Q, "*:*"); solrParams.set(ShardParams._ROUTE_, sameShardRoutes.get(random().nextInt(sameShardRoutes.size()))); - log.info("output : {}" ,cloudClient.query(solrParams)); + log.info("output: {}", cluster.getSolrClient().query(COLLECTION, solrParams)); } // Request counts increase from expected nodes should aggregate to 1000, while there should be @@ -341,7 +277,7 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { String baseURL = (String) replica.get(ZkStateReader.BASE_URL_PROP); Long prevNumRequests = requestCountsMap.get(baseURL); - Long curNumRequests = getNumRequests(baseURL, collectionName); + Long curNumRequests = getNumRequests(baseURL, COLLECTION); long delta = curNumRequests - prevNumRequests; if (expectedBaseURLs.contains(baseURL)) { @@ -357,74 +293,36 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { assertEquals("Unexpected number of requests to unexpected URLs: " + numRequestsToUnexpectedUrls, 0, increaseFromUnexpectedUrls); - controlClient.deleteByQuery("*:*"); - cloudClient.deleteByQuery("*:*"); - - controlClient.commit(); - cloudClient.commit(); - cloudClient.close(); } /** * Tests if the specification of 'preferLocalShards' in the query-params * limits the distributed query to locally hosted shards only */ - private void preferLocalShardsTest() throws Exception { + @Test + public void preferLocalShardsTest() throws Exception { String collectionName = "localShardsTestColl"; - int liveNodes = getCommonCloudSolrClient() - .getZkStateReader().getClusterState().getLiveNodes().size(); + int liveNodes = cluster.getJettySolrRunners().size(); // For preferLocalShards to succeed in a test, every shard should have // all its cores on the same node. // Hence the below configuration for our collection - Map props = makeMap( - REPLICATION_FACTOR, liveNodes, - MAX_SHARDS_PER_NODE, liveNodes, - NUM_SLICES, liveNodes); - Map> collectionInfos = new HashMap>(); - createCollection(collectionInfos, collectionName, props, controlClientCloud); - waitForRecoveriesToFinish(collectionName, false); - - CloudSolrClient cloudClient = createCloudClient(collectionName); - assertNotNull(cloudClient); - handle.clear(); - handle.put("timestamp", SKIPVAL); - waitForThingsToLevelOut(30); - - // Remove any documents from previous test (if any) - controlClient.deleteByQuery("*:*"); - cloudClient.deleteByQuery("*:*"); - controlClient.commit(); - cloudClient.commit(); + CollectionAdminRequest.createCollection(collectionName, "conf", liveNodes, liveNodes) + .setMaxShardsPerNode(liveNodes) + .processAndWait(cluster.getSolrClient(), TIMEOUT); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, cluster.getSolrClient().getZkStateReader(), false, true, TIMEOUT); // Add some new documents - SolrInputDocument doc1 = new SolrInputDocument(); - doc1.addField(id, "0"); - doc1.addField("a_t", "hello1"); - SolrInputDocument doc2 = new SolrInputDocument(); - doc2.addField(id, "2"); - doc2.addField("a_t", "hello2"); - SolrInputDocument doc3 = new SolrInputDocument(); - doc3.addField(id, "3"); - doc3.addField("a_t", "hello2"); - - UpdateRequest request = new UpdateRequest(); - request.add(doc1); - request.add(doc2); - request.add(doc3); - request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false); + new UpdateRequest() + .add(id, "0", "a_t", "hello1") + .add(id, "2", "a_t", "hello2") + .add(id, "3", "a_t", "hello2") + .commit(cluster.getSolrClient(), collectionName); // Run the actual test for 'preferLocalShards' - queryWithPreferLocalShards(cloudClient, true, collectionName); - - // Cleanup - controlClient.deleteByQuery("*:*"); - cloudClient.deleteByQuery("*:*"); - controlClient.commit(); - cloudClient.commit(); - cloudClient.close(); + queryWithPreferLocalShards(cluster.getSolrClient(), true, collectionName); } private void queryWithPreferLocalShards(CloudSolrClient cloudClient, @@ -432,8 +330,7 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { String collectionName) throws Exception { - SolrQuery qRequest = new SolrQuery(); - qRequest.setQuery("*:*"); + SolrQuery qRequest = new SolrQuery("*:*"); ModifiableSolrParams qParams = new ModifiableSolrParams(); qParams.add("preferLocalShards", Boolean.toString(preferLocalShards)); @@ -444,7 +341,7 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { // And since all the nodes are hosting cores from all shards, the // distributed query formed by this node will select cores from the // local shards only - QueryResponse qResponse = cloudClient.query (qRequest); + QueryResponse qResponse = cloudClient.query(collectionName, qRequest); Object shardsInfo = qResponse.getResponse().get(ShardParams.SHARDS_INFO); assertNotNull("Unable to obtain "+ShardParams.SHARDS_INFO, shardsInfo); @@ -495,21 +392,23 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { return (Long) resp.findRecursive("solr-mbeans", "QUERYHANDLER", "standard", "stats", "requests"); } - - @Override - protected void indexr(Object... fields) throws Exception { - SolrInputDocument doc = getDoc(fields); - indexDoc(doc); - } - private void checkCollectionParameters() throws Exception { + @Test + public void checkCollectionParameters() throws Exception { - try (CloudSolrClient client = createCloudClient("multicollection1")) { + try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress())) { - createCollection("multicollection1", client, 2, 2); - createCollection("multicollection2", client, 2, 2); - waitForRecoveriesToFinish("multicollection1", false); - waitForRecoveriesToFinish("multicollection2", false); + String async1 = CollectionAdminRequest.createCollection("multicollection1", "conf", 2, 1) + .processAsync(client); + String async2 = CollectionAdminRequest.createCollection("multicollection2", "conf", 2, 1) + .processAsync(client); + + CollectionAdminRequest.waitForAsyncRequest(async1, client, TIMEOUT); + CollectionAdminRequest.waitForAsyncRequest(async2, client, TIMEOUT); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("multicollection1", client.getZkStateReader(), false, true, TIMEOUT); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("multicollection2", client.getZkStateReader(), false, true, TIMEOUT); + + client.setDefaultCollection("multicollection1"); List docs = new ArrayList<>(3); for (int i = 0; i < 3; i++) { @@ -540,73 +439,70 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { } - private void stateVersionParamTest() throws Exception { + @Test + public void stateVersionParamTest() throws Exception { - try (CloudSolrClient client = createCloudClient(null)) { - String collectionName = "checkStateVerCol"; - createCollection(collectionName, client, 1, 3); - waitForRecoveriesToFinish(collectionName, false); - DocCollection coll = client.getZkStateReader().getClusterState().getCollection(collectionName); - Replica r = coll.getSlices().iterator().next().getReplicas().iterator().next(); + DocCollection coll = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION); + Replica r = coll.getSlices().iterator().next().getReplicas().iterator().next(); - SolrQuery q = new SolrQuery().setQuery("*:*"); - HttpSolrClient.RemoteSolrException sse = null; + SolrQuery q = new SolrQuery().setQuery("*:*"); + HttpSolrClient.RemoteSolrException sse = null; - final String url = r.getStr(ZkStateReader.BASE_URL_PROP) + "/" +collectionName; - try (HttpSolrClient solrClient = getHttpSolrClient(url)) { + final String url = r.getStr(ZkStateReader.BASE_URL_PROP) + "/" + COLLECTION; + try (HttpSolrClient solrClient = getHttpSolrClient(url)) { - log.info("should work query, result {}", solrClient.query(q)); - //no problem - q.setParam(CloudSolrClient.STATE_VERSION, collectionName + ":" + coll.getZNodeVersion()); - log.info("2nd query , result {}", solrClient.query(q)); - //no error yet good + log.info("should work query, result {}", solrClient.query(q)); + //no problem + q.setParam(CloudSolrClient.STATE_VERSION, COLLECTION + ":" + coll.getZNodeVersion()); + log.info("2nd query , result {}", solrClient.query(q)); + //no error yet good - q.setParam(CloudSolrClient.STATE_VERSION, collectionName + ":" + (coll.getZNodeVersion() - 1)); //an older version expect error + q.setParam(CloudSolrClient.STATE_VERSION, COLLECTION + ":" + (coll.getZNodeVersion() - 1)); //an older version expect error + QueryResponse rsp = solrClient.query(q); + Map m = (Map) rsp.getResponse().get(CloudSolrClient.STATE_VERSION, rsp.getResponse().size()-1); + assertNotNull("Expected an extra information from server with the list of invalid collection states", m); + assertNotNull(m.get(COLLECTION)); + } + + //now send the request to another node that does not serve the collection + + Set allNodesOfColl = new HashSet<>(); + for (Slice slice : coll.getSlices()) { + for (Replica replica : slice.getReplicas()) { + allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP)); + } + } + String theNode = null; + Set liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes(); + for (String s : liveNodes) { + String n = cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(s); + if(!allNodesOfColl.contains(n)){ + theNode = n; + break; + } + } + log.info("the node which does not serve this collection{} ",theNode); + assertNotNull(theNode); + + + final String solrClientUrl = theNode + "/" + COLLECTION; + try (SolrClient solrClient = getHttpSolrClient(solrClientUrl)) { + + q.setParam(CloudSolrClient.STATE_VERSION, COLLECTION + ":" + (coll.getZNodeVersion()-1)); + try { QueryResponse rsp = solrClient.query(q); - Map m = (Map) rsp.getResponse().get(CloudSolrClient.STATE_VERSION, rsp.getResponse().size()-1); - assertNotNull("Expected an extra information from server with the list of invalid collection states", m); - assertNotNull(m.get(collectionName)); - } - - //now send the request to another node that does not serve the collection - - Set allNodesOfColl = new HashSet<>(); - for (Slice slice : coll.getSlices()) { - for (Replica replica : slice.getReplicas()) { - allNodesOfColl.add(replica.getStr(ZkStateReader.BASE_URL_PROP)); - } - } - String theNode = null; - Set liveNodes = client.getZkStateReader().getClusterState().getLiveNodes(); - for (String s : liveNodes) { - String n = client.getZkStateReader().getBaseUrlForNodeName(s); - if(!allNodesOfColl.contains(n)){ - theNode = n; - break; - } - } - log.info("the node which does not serve this collection{} ",theNode); - assertNotNull(theNode); - - - final String solrClientUrl = theNode + "/" + collectionName; - try (SolrClient solrClient = getHttpSolrClient(solrClientUrl)) { - - q.setParam(CloudSolrClient.STATE_VERSION, collectionName + ":" + (coll.getZNodeVersion()-1)); - try { - QueryResponse rsp = solrClient.query(q); - log.info("error was expected"); - } catch (HttpSolrClient.RemoteSolrException e) { - sse = e; - } - assertNotNull(sse); - assertEquals(" Error code should be 510", SolrException.ErrorCode.INVALID_STATE.code, sse.code()); + log.info("error was expected"); + } catch (HttpSolrClient.RemoteSolrException e) { + sse = e; } + assertNotNull(sse); + assertEquals(" Error code should be 510", SolrException.ErrorCode.INVALID_STATE.code, sse.code()); } } + @Test public void testShutdown() throws IOException { try (CloudSolrClient client = getCloudSolrClient("[ff01::114]:33332")) { client.setZkConnectTimeout(100); @@ -620,22 +516,23 @@ public class CloudSolrClientTest extends AbstractFullDistribZkTestBase { @Rule public ExpectedException exception = ExpectedException.none(); + @Test public void testWrongZkChrootTest() throws IOException { exception.expect(SolrException.class); exception.expectMessage("cluster not found/not ready"); - try (CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress() + "/xyz/foo")) { - client.setDefaultCollection(DEFAULT_COLLECTION); + try (CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress() + "/xyz/foo")) { client.setZkClientTimeout(1000 * 60); client.connect(); fail("Expected exception"); } } + @Test public void customHttpClientTest() throws IOException { CloseableHttpClient client = HttpClientUtil.createClient(null); - try (CloudSolrClient solrClient = getCloudSolrClient(zkServer.getZkAddress(), client)) { + try (CloudSolrClient solrClient = getCloudSolrClient(cluster.getZkServer().getZkAddress(), client)) { assertTrue(solrClient.getLbClient().getHttpClient() == client); diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java index 7c1f97d452d..53f71265910 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java @@ -17,14 +17,12 @@ package org.apache.solr.client.solrj.io.graph; * limitations under the License. */ -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Set; @@ -34,18 +32,20 @@ import org.apache.solr.client.solrj.io.SolrClientCache; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; import org.apache.solr.client.solrj.io.comp.FieldComparator; -import org.apache.solr.client.solrj.io.stream.*; +import org.apache.solr.client.solrj.io.stream.CloudSolrStream; +import org.apache.solr.client.solrj.io.stream.HashJoinStream; +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.StreamFactory; import org.apache.solr.client.solrj.io.stream.metrics.CountMetric; import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric; import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric; import org.apache.solr.client.solrj.io.stream.metrics.MinMetric; import org.apache.solr.client.solrj.io.stream.metrics.SumMetric; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.common.SolrInputDocument; -import org.junit.After; -import org.junit.AfterClass; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -58,96 +58,52 @@ import org.junit.Test; @Slow @LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"}) -public class GraphExpressionTest extends AbstractFullDistribZkTestBase { +public class GraphExpressionTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); + private static final String COLLECTION = "collection1"; - static { - schemaString = "schema-streaming.xml"; - } + private static final String id = "id"; + + private static final int TIMEOUT = 30; @BeforeClass - public static void beforeSuperClass() { - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME()); - } + public static void setupCluster() throws Exception { + configureCluster(2) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); - @AfterClass - public static void afterSuperClass() { - - } - - protected String getCloudSolrConfig() { - return "solrconfig-streaming.xml"; - } - - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - public static String SOLR_HOME() { - return SOLR_HOME; + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); } @Before - @Override - public void setUp() throws Exception { - super.setUp(); - // we expect this time of exception as shards go up and down... - //ignoreException(".*"); - - System.setProperty("numShards", Integer.toString(sliceCount)); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - resetExceptionIgnores(); - } - - public GraphExpressionTest() { - super(); - sliceCount = 2; + public void cleanIndex() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); } @Test - public void testAll() throws Exception{ - assertNotNull(cloudClient); + public void testShortestPathStream() throws Exception { - handle.clear(); - handle.put("timestamp", SKIPVAL); - - waitForRecoveriesToFinish(false); - - del("*:*"); - commit(); - - testShortestPathStream(); - testGatherNodesStream(); - testGatherNodesFriendsStream(); - } - - private void testShortestPathStream() throws Exception { - - indexr(id, "0", "from_s", "jim", "to_s", "mike", "predicate_s", "knows"); - indexr(id, "1", "from_s", "jim", "to_s", "dave", "predicate_s", "knows"); - indexr(id, "2", "from_s", "jim", "to_s", "stan", "predicate_s", "knows"); - indexr(id, "3", "from_s", "dave", "to_s", "stan", "predicate_s", "knows"); - indexr(id, "4", "from_s", "dave", "to_s", "bill", "predicate_s", "knows"); - indexr(id, "5", "from_s", "dave", "to_s", "mike", "predicate_s", "knows"); - indexr(id, "20", "from_s", "dave", "to_s", "alex", "predicate_s", "knows"); - indexr(id, "21", "from_s", "alex", "to_s", "steve", "predicate_s", "knows"); - indexr(id, "6", "from_s", "stan", "to_s", "alice", "predicate_s", "knows"); - indexr(id, "7", "from_s", "stan", "to_s", "mary", "predicate_s", "knows"); - indexr(id, "8", "from_s", "stan", "to_s", "dave", "predicate_s", "knows"); - indexr(id, "10", "from_s", "mary", "to_s", "mike", "predicate_s", "knows"); - indexr(id, "11", "from_s", "mary", "to_s", "max", "predicate_s", "knows"); - indexr(id, "12", "from_s", "mary", "to_s", "jim", "predicate_s", "knows"); - indexr(id, "13", "from_s", "mary", "to_s", "steve", "predicate_s", "knows"); - - commit(); + new UpdateRequest() + .add(id, "0", "from_s", "jim", "to_s", "mike", "predicate_s", "knows") + .add(id, "1", "from_s", "jim", "to_s", "dave", "predicate_s", "knows") + .add(id, "2", "from_s", "jim", "to_s", "stan", "predicate_s", "knows") + .add(id, "3", "from_s", "dave", "to_s", "stan", "predicate_s", "knows") + .add(id, "4", "from_s", "dave", "to_s", "bill", "predicate_s", "knows") + .add(id, "5", "from_s", "dave", "to_s", "mike", "predicate_s", "knows") + .add(id, "20", "from_s", "dave", "to_s", "alex", "predicate_s", "knows") + .add(id, "21", "from_s", "alex", "to_s", "steve", "predicate_s", "knows") + .add(id, "6", "from_s", "stan", "to_s", "alice", "predicate_s", "knows") + .add(id, "7", "from_s", "stan", "to_s", "mary", "predicate_s", "knows") + .add(id, "8", "from_s", "stan", "to_s", "dave", "predicate_s", "knows") + .add(id, "10", "from_s", "mary", "to_s", "mike", "predicate_s", "knows") + .add(id, "11", "from_s", "mary", "to_s", "max", "predicate_s", "knows") + .add(id, "12", "from_s", "mary", "to_s", "jim", "predicate_s", "knows") + .add(id, "13", "from_s", "mary", "to_s", "steve", "predicate_s", "knows") + .commit(cluster.getSolrClient(), COLLECTION); List tuples = null; Set paths = null; @@ -157,7 +113,7 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { context.setSolrClientCache(cache); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("shortestPath", ShortestPathStream.class); Map params = new HashMap(); @@ -271,27 +227,26 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(paths.contains("[jim, stan, mary, steve]")); cache.close(); - del("*:*"); - commit(); + } + @Test + public void testGatherNodesStream() throws Exception { - private void testGatherNodesStream() throws Exception { - - indexr(id, "0", "basket_s", "basket1", "product_s", "product1", "price_f", "20"); - indexr(id, "1", "basket_s", "basket1", "product_s", "product3", "price_f", "30"); - indexr(id, "2", "basket_s", "basket1", "product_s", "product5", "price_f", "1"); - indexr(id, "3", "basket_s", "basket2", "product_s", "product1", "price_f", "2"); - indexr(id, "4", "basket_s", "basket2", "product_s", "product6", "price_f", "5"); - indexr(id, "5", "basket_s", "basket2", "product_s", "product7", "price_f", "10"); - indexr(id, "6", "basket_s", "basket3", "product_s", "product4", "price_f", "20"); - indexr(id, "7", "basket_s", "basket3", "product_s", "product3", "price_f", "10"); - indexr(id, "8", "basket_s", "basket3", "product_s", "product1", "price_f", "10"); - indexr(id, "9", "basket_s", "basket4", "product_s", "product4", "price_f", "40"); - indexr(id, "10", "basket_s", "basket4", "product_s", "product3", "price_f", "10"); - indexr(id, "11", "basket_s", "basket4", "product_s", "product1", "price_f", "10"); - - commit(); + new UpdateRequest() + .add(id, "0", "basket_s", "basket1", "product_s", "product1", "price_f", "20") + .add(id, "1", "basket_s", "basket1", "product_s", "product3", "price_f", "30") + .add(id, "2", "basket_s", "basket1", "product_s", "product5", "price_f", "1") + .add(id, "3", "basket_s", "basket2", "product_s", "product1", "price_f", "2") + .add(id, "4", "basket_s", "basket2", "product_s", "product6", "price_f", "5") + .add(id, "5", "basket_s", "basket2", "product_s", "product7", "price_f", "10") + .add(id, "6", "basket_s", "basket3", "product_s", "product4", "price_f", "20") + .add(id, "7", "basket_s", "basket3", "product_s", "product3", "price_f", "10") + .add(id, "8", "basket_s", "basket3", "product_s", "product1", "price_f", "10") + .add(id, "9", "basket_s", "basket4", "product_s", "product4", "price_f", "40") + .add(id, "10", "basket_s", "basket4", "product_s", "product3", "price_f", "10") + .add(id, "11", "basket_s", "basket4", "product_s", "product1", "price_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); List tuples = null; Set paths = null; @@ -301,7 +256,7 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { context.setSolrClientCache(cache); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("gatherNodes", GatherNodesStream.class) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("count", CountMetric.class) @@ -417,20 +372,20 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(tuples.get(1).getString("node").equals("basket3")); cache.close(); - del("*:*"); - commit(); + } - private void testGatherNodesFriendsStream() throws Exception { + @Test + public void testGatherNodesFriendsStream() throws Exception { - indexr(id, "0", "from_s", "bill", "to_s", "jim", "message_t", "Hello jim"); - indexr(id, "1", "from_s", "bill", "to_s", "sam", "message_t", "Hello sam"); - indexr(id, "2", "from_s", "bill", "to_s", "max", "message_t", "Hello max"); - indexr(id, "3", "from_s", "max", "to_s", "kip", "message_t", "Hello kip"); - indexr(id, "4", "from_s", "sam", "to_s", "steve", "message_t", "Hello steve"); - indexr(id, "5", "from_s", "jim", "to_s", "ann", "message_t", "Hello steve"); - - commit(); + new UpdateRequest() + .add(id, "0", "from_s", "bill", "to_s", "jim", "message_t", "Hello jim") + .add(id, "1", "from_s", "bill", "to_s", "sam", "message_t", "Hello sam") + .add(id, "2", "from_s", "bill", "to_s", "max", "message_t", "Hello max") + .add(id, "3", "from_s", "max", "to_s", "kip", "message_t", "Hello kip") + .add(id, "4", "from_s", "sam", "to_s", "steve", "message_t", "Hello steve") + .add(id, "5", "from_s", "jim", "to_s", "ann", "message_t", "Hello steve") + .commit(cluster.getSolrClient(), COLLECTION); List tuples = null; Set paths = null; @@ -440,7 +395,7 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { context.setSolrClientCache(cache); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("gatherNodes", GatherNodesStream.class) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("count", CountMetric.class) @@ -628,10 +583,10 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(tuples.get(6).getLong("level").equals(new Long(2))); //Add a cycle from jim to bill - indexr(id, "6", "from_s", "jim", "to_s", "bill", "message_t", "Hello steve"); - indexr(id, "7", "from_s", "sam", "to_s", "bill", "message_t", "Hello steve"); - - commit(); + new UpdateRequest() + .add(id, "6", "from_s", "jim", "to_s", "bill", "message_t", "Hello steve") + .add(id, "7", "from_s", "sam", "to_s", "bill", "message_t", "Hello steve") + .commit(cluster.getSolrClient(), COLLECTION); expr = "gatherNodes(collection1, " + "search(collection1, q=\"message_t:jim\", fl=\"from_s\", sort=\"from_s asc\"),"+ @@ -676,12 +631,9 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(tuples.get(6).getLong("level").equals(new Long(2))); cache.close(); - del("*:*"); - commit(); + } - - protected List getTuples(TupleStream tupleStream) throws IOException { tupleStream.open(); List tuples = new ArrayList(); @@ -691,9 +643,7 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { tupleStream.close(); return tuples; } - protected boolean assertOrder(List tuples, int... ids) throws Exception { - return assertOrderOf(tuples, "id", ids); - } + protected boolean assertOrderOf(List tuples, String fieldName, int... ids) throws Exception { int i = 0; for(int val : ids) { @@ -707,56 +657,6 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { return true; } - protected boolean assertMapOrder(List tuples, int... ids) throws Exception { - int i = 0; - for(int val : ids) { - Tuple t = tuples.get(i); - List tip = t.getMaps("group"); - int id = (int)tip.get(0).get("id"); - if(id != val) { - throw new Exception("Found value:"+id+" expecting:"+val); - } - ++i; - } - return true; - } - - - protected boolean assertFields(List tuples, String ... fields) throws Exception{ - for(Tuple tuple : tuples){ - for(String field : fields){ - if(!tuple.fields.containsKey(field)){ - throw new Exception(String.format(Locale.ROOT, "Expected field '%s' not found", field)); - } - } - } - return true; - } - protected boolean assertNotFields(List tuples, String ... fields) throws Exception{ - for(Tuple tuple : tuples){ - for(String field : fields){ - if(tuple.fields.containsKey(field)){ - throw new Exception(String.format(Locale.ROOT, "Unexpected field '%s' found", field)); - } - } - } - return true; - } - - protected boolean assertGroupOrder(Tuple tuple, int... ids) throws Exception { - List group = (List)tuple.get("tuples"); - int i=0; - for(int val : ids) { - Map t = (Map)group.get(i); - Long tip = (Long)t.get("id"); - if(tip.intValue() != val) { - throw new Exception("Found value:"+tip.intValue()+" expecting:"+val); - } - ++i; - } - return true; - } - public boolean assertLong(Tuple tuple, String fieldName, long l) throws Exception { long lv = (long)tuple.get(fieldName); if(lv != l) { @@ -778,44 +678,4 @@ public class GraphExpressionTest extends AbstractFullDistribZkTestBase { return true; } - protected boolean assertMaps(List maps, int... ids) throws Exception { - if(maps.size() != ids.length) { - throw new Exception("Expected id count != actual map count:"+ids.length+":"+maps.size()); - } - - int i=0; - for(int val : ids) { - Map t = maps.get(i); - Long tip = (Long)t.get("id"); - if(tip.intValue() != val) { - throw new Exception("Found value:"+tip.intValue()+" expecting:"+val); - } - ++i; - } - return true; - } - - private boolean assertList(List list, Object... vals) throws Exception { - - if(list.size() != vals.length) { - throw new Exception("Lists are not the same size:"+list.size() +" : "+vals.length); - } - - for(int i=0; i tuples = null; Set paths = null; ShortestPathStream stream = null; - String zkHost = zkServer.getZkAddress(); + String zkHost = cluster.getZkServer().getZkAddress(); StreamContext context = new StreamContext(); SolrClientCache cache = new SolrClientCache(); context.setSolrClientCache(cache); @@ -260,40 +227,6 @@ public class GraphTest extends AbstractFullDistribZkTestBase { assertTrue(paths.contains("[jim, stan, mary, steve]")); cache.close(); - del("*:*"); - commit(); - } - - @Test - public void streamTests() throws Exception { - assertNotNull(cloudClient); - - handle.clear(); - handle.put("timestamp", SKIPVAL); - - waitForRecoveriesToFinish(false); - - del("*:*"); - - commit(); - - testShortestPathStream(); - - } - - protected Map mapParams(String... vals) { - Map params = new HashMap(); - String k = null; - for(String val : vals) { - if(k == null) { - k = val; - } else { - params.put(k, val); - k = null; - } - } - - return params; } protected List getTuples(TupleStream tupleStream) throws IOException { @@ -311,58 +244,6 @@ public class GraphTest extends AbstractFullDistribZkTestBase { return tuples; } - protected Tuple getTuple(TupleStream tupleStream) throws IOException { - tupleStream.open(); - Tuple t = tupleStream.read(); - tupleStream.close(); - return t; - } - - - protected boolean assertOrder(List tuples, int... ids) throws Exception { - int i = 0; - for(int val : ids) { - Tuple t = tuples.get(i); - Long tip = (Long)t.get("id"); - if(tip.intValue() != val) { - throw new Exception("Found value:"+tip.intValue()+" expecting:"+val); - } - ++i; - } - return true; - } - - protected boolean assertGroupOrder(Tuple tuple, int... ids) throws Exception { - List group = (List)tuple.get("tuples"); - int i=0; - for(int val : ids) { - Map t = (Map)group.get(i); - Long tip = (Long)t.get("id"); - if(tip.intValue() != val) { - throw new Exception("Found value:"+tip.intValue()+" expecting:"+val); - } - ++i; - } - return true; - } - - protected boolean assertMaps(List maps, int... ids) throws Exception { - if(maps.size() != ids.length) { - throw new Exception("Expected id count != actual map count:"+ids.length+":"+maps.size()); - } - - int i=0; - for(int val : ids) { - Map t = maps.get(i); - Long tip = (Long)t.get("id"); - if(tip.intValue() != val) { - throw new Exception("Found value:"+tip.intValue()+" expecting:"+val); - } - ++i; - } - return true; - } - public boolean assertLong(Tuple tuple, String fieldName, long l) throws Exception { long lv = (long)tuple.get(fieldName); if(lv != l) { @@ -372,16 +253,5 @@ public class GraphTest extends AbstractFullDistribZkTestBase { return true; } - @Override - protected void indexr(Object... fields) throws Exception { - SolrInputDocument doc = getDoc(fields); - indexDoc(doc); - } - - private void attachStreamFactory(TupleStream tupleStream) { - StreamContext streamContext = new StreamContext(); - streamContext.setStreamFactory(streamFactory); - tupleStream.setStreamContext(streamContext); - } } diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java index d38661e9e95..1829c70f54a 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java @@ -16,7 +16,6 @@ */ package org.apache.solr.client.solrj.io.sql; -import java.io.File; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -32,11 +31,10 @@ 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.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.junit.BeforeClass; import org.junit.Test; @@ -47,68 +45,45 @@ import org.junit.Test; @Slow @LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40", "Lucene41", "Lucene42", "Lucene45"}) -public class JdbcTest extends AbstractFullDistribZkTestBase { +public class JdbcTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); + private static final String COLLECTION = "collection1"; + private static final String id = "id"; - static { - schemaString = "schema-sql.xml"; - } + private static final int TIMEOUT = 30; + + private static String zkHost; @BeforeClass - public static void beforeSuperClass() { - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME); - } + public static void setupCluster() throws Exception { + configureCluster(2) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); - @AfterClass - public static void afterSuperClass() { + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); - } + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "testnull_i", null) + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "testnull_i", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "testnull_i", null) + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "testnull_i", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "testnull_i", null) + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6", "testnull_i", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7", "testnull_i", null) + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8", "testnull_i", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9", "testnull_i", null) + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10", "testnull_i", "10") + .commit(cluster.getSolrClient(), COLLECTION); - protected String getCloudSolrConfig() { - return "solrconfig-sql.xml"; - } - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - - @Override - public void distribSetUp() throws Exception { - super.distribSetUp(); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - resetExceptionIgnores(); + zkHost = cluster.getZkServer().getZkAddress(); } @Test - @ShardsFixed(num = 2) public void doTest() throws Exception { - waitForRecoveriesToFinish(false); - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "testnull_i", null); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "testnull_i", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "testnull_i", null); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "testnull_i", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "testnull_i", null); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6", "testnull_i", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7", "testnull_i", null); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8", "testnull_i", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9", "testnull_i", null); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10", "testnull_i", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - Properties props = new Properties(); try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) { @@ -202,8 +177,13 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } + } + + @Test + public void testFacetAggregation() throws Exception { + //Test facet aggregation - props = new Properties(); + Properties props = new Properties(); props.put("aggregationMode", "facet"); try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) { try (Statement stmt = con.createStatement()) { @@ -236,8 +216,13 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } + } + + @Test + public void testMapReduceAggregation() throws Exception { + //Test map / reduce aggregation - props = new Properties(); + Properties props = new Properties(); props.put("aggregationMode", "map_reduce"); props.put("numWorkers", "2"); try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) { @@ -270,15 +255,20 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } } - + + } + + @Test + public void testConnectionParams() throws Exception { + //Test params on the url - try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + + try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1&aggregationMode=map_reduce&numWorkers=2")) { Properties p = ((ConnectionImpl) con).getProperties(); - assert(p.getProperty("aggregationMode").equals("map_reduce")); - assert(p.getProperty("numWorkers").equals("2")); + assert (p.getProperty("aggregationMode").equals("map_reduce")); + assert (p.getProperty("numWorkers").equals("2")); try (Statement stmt = con.createStatement()) { try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " + @@ -310,6 +300,11 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } + } + + @Test + public void testJDBCUrlParameters() throws Exception { + // Test JDBC paramters in URL try (Connection con = DriverManager.getConnection( "jdbc:solr://" + zkHost + "?collection=collection1&username=&password=&testKey1=testValue&testKey2")) { @@ -350,6 +345,11 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } + } + + @Test + public void testJDBCPropertiesParameters() throws Exception { + // Test JDBC paramters in properties Properties providedProperties = new Properties(); providedProperties.put("collection", "collection1"); @@ -360,10 +360,10 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost, providedProperties)) { Properties p = ((ConnectionImpl) con).getProperties(); - assert(p.getProperty("username").equals("")); - assert(p.getProperty("password").equals("")); - assert(p.getProperty("testKey1").equals("testValue")); - assert(p.getProperty("testKey2").equals("")); + assert (p.getProperty("username").equals("")); + assert (p.getProperty("password").equals("")); + assert (p.getProperty("testKey1").equals("testValue")); + assert (p.getProperty("testKey2").equals("")); try (Statement stmt = con.createStatement()) { try (ResultSet rs = stmt.executeQuery("select a_s, sum(a_f) from collection1 group by a_s " + @@ -394,10 +394,13 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } } + } + @Test + public void testErrorPropagation() throws Exception { //Test error propagation - props = new Properties(); + Properties props = new Properties(); props.put("aggregationMode", "facet"); try (Connection con = DriverManager.getConnection("jdbc:solr://" + zkHost + "?collection=collection1", props)) { try (Statement stmt = con.createStatement()) { @@ -410,20 +413,20 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { } } - testDriverMetadata(); } - private void testDriverMetadata() throws Exception { - String collection = DEFAULT_COLLECTION; + @Test + public void testDriverMetadata() throws Exception { + String collection = COLLECTION; - String connectionString1 = "jdbc:solr://" + zkServer.getZkAddress() + "?collection=" + collection + + String connectionString1 = "jdbc:solr://" + zkHost + "?collection=" + collection + "&username=&password=&testKey1=testValue&testKey2"; Properties properties1 = new Properties(); String sql = "select id, a_i, a_s, a_f as my_float_col, testnull_i from " + collection + " order by a_i desc"; - String connectionString2 = "jdbc:solr://" + zkServer.getZkAddress() + "?collection=" + collection + + String connectionString2 = "jdbc:solr://" + zkHost + "?collection=" + collection + "&aggregationMode=map_reduce&numWorkers=2&username=&password=&testKey1=testValue&testKey2"; Properties properties2 = new Properties(); @@ -439,9 +442,9 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { try (Connection con = DriverManager.getConnection(connectionString, properties)) { assertTrue(con.isValid(DEFAULT_CONNECTION_TIMEOUT)); - assertEquals(zkServer.getZkAddress(), con.getCatalog()); - con.setCatalog(zkServer.getZkAddress()); - assertEquals(zkServer.getZkAddress(), con.getCatalog()); + assertEquals(zkHost, con.getCatalog()); + con.setCatalog(zkHost); + assertEquals(zkHost, con.getCatalog()); assertEquals(null, con.getSchema()); con.setSchema("myschema"); @@ -470,22 +473,22 @@ public class JdbcTest extends AbstractFullDistribZkTestBase { try(ResultSet rs = databaseMetaData.getCatalogs()) { assertTrue(rs.next()); - assertEquals(zkServer.getZkAddress(), rs.getString("TABLE_CAT")); + assertEquals(zkHost, rs.getString("TABLE_CAT")); assertFalse(rs.next()); } List collections = new ArrayList<>(); - collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollections()); + collections.addAll(cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionsMap().keySet()); Collections.sort(collections); try(ResultSet rs = databaseMetaData.getSchemas()) { assertFalse(rs.next()); } - try(ResultSet rs = databaseMetaData.getTables(zkServer.getZkAddress(), null, "%", null)) { + try(ResultSet rs = databaseMetaData.getTables(zkHost, null, "%", null)) { for(String acollection : collections) { assertTrue(rs.next()); - assertEquals(zkServer.getZkAddress(), rs.getString("TABLE_CAT")); + assertEquals(zkHost, rs.getString("TABLE_CAT")); assertNull(rs.getString("TABLE_SCHEM")); assertEquals(acollection, rs.getString("TABLE_NAME")); assertEquals("TABLE", rs.getString("TABLE_TYPE")); diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java index f330c110a9e..0315cfec0e1 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/JDBCStreamTest.java @@ -16,7 +16,6 @@ */ package org.apache.solr.client.solrj.io.stream; -import java.io.File; import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; @@ -26,10 +25,8 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Locale; -import java.util.Properties; import org.apache.lucene.util.LuceneTestCase; -import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; import org.apache.solr.client.solrj.io.comp.FieldComparator; @@ -38,10 +35,10 @@ import org.apache.solr.client.solrj.io.stream.metrics.CountMetric; import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric; import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric; import org.apache.solr.client.solrj.io.stream.metrics.MinMetric; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.common.SolrInputDocument; -import org.junit.After; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -50,19 +47,28 @@ import org.junit.Test; /** */ -@Slow @LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"}) -public class JDBCStreamTest extends AbstractFullDistribZkTestBase { +public class JDBCStreamTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); + private static final String COLLECTION = "jdbc"; - static { - schemaString = "schema-streaming.xml"; + private static final int TIMEOUT = 30; + + private static final String id = "id"; + + @BeforeClass + public static void setupCluster() throws Exception { + configureCluster(4) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); + + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); } @BeforeClass - public static void beforeSuperClass() throws Exception { - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME()); + public static void setupDatabase() throws Exception { // Initialize Database // Ok, so.....hsqldb is doing something totally weird so I thought I'd take a moment to explain it. @@ -74,8 +80,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { // JDBCStream and is only a carryover from the driver we are testing with. Class.forName("org.hsqldb.jdbcDriver").newInstance(); Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement = connection.createStatement(); + Statement statement = connection.createStatement(); statement.executeUpdate("create table COUNTRIES(CODE varchar(3) not null primary key, COUNTRY_NAME varchar(50), DELETED char(1) default 'N')"); statement.executeUpdate("create table PEOPLE(ID int not null primary key, NAME varchar(50), COUNTRY_CODE char(2), DELETED char(1) default 'N')"); statement.executeUpdate("create table PEOPLE_SPORTS(ID int not null primary key, PERSON_ID int, SPORT_NAME varchar(50), DELETED char(1) default 'N')"); @@ -83,107 +88,48 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { } @AfterClass - public static void afterSuperClass() throws SQLException { + public static void teardownDatabase() throws SQLException { Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); Statement statement = connection.createStatement(); statement.executeUpdate("shutdown"); } - protected String getCloudSolrConfig() { - return "solrconfig-streaming.xml"; - } - - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - public static String SOLR_HOME() { - return SOLR_HOME; + @Before + public void cleanIndex() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); } @Before - @Override - public void setUp() throws Exception { - super.setUp(); - // we expect this time of exception as shards go up and down... - //ignoreException(".*"); - - System.setProperty("numShards", Integer.toString(sliceCount)); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - resetExceptionIgnores(); - } - - public JDBCStreamTest() { - super(); - sliceCount = 2; + public void cleanDatabase() throws Exception { + // Clear database + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("delete from COUNTRIES WHERE 1=1"); + statement.executeUpdate("delete from PEOPLE WHERE 1=1"); + statement.executeUpdate("delete from PEOPLE_SPORTS WHERE 1=1"); + } } @Test - public void testAll() throws Exception{ - assertNotNull(cloudClient); + public void testJDBCSelect() throws Exception { - handle.clear(); - handle.put("timestamp", SKIPVAL); - - waitForRecoveriesToFinish(false); - - // Run JDBC Only tests - testJDBCSelect(); - testJDBCJoin(); - - // Run JDBC + Solr tests - testJDBCSolrMerge(); - testJDBCSolrInnerJoinExpression(); - testJDBCSolrInnerJoinRollupExpression(); - testJDBCSolrInnerJoinExpressionWithProperties(); - - // Clear all data - clearData(); - - // Delete database - // done during afterSuperClass(...) - } - - private void clearData() throws Exception { - // Clear Solr index - del("*:*"); - commit(); - - // Clear database - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("delete from COUNTRIES WHERE 1=1"); - statement.executeUpdate("delete from PEOPLE WHERE 1=1"); - statement.executeUpdate("delete from PEOPLE_SPORTS WHERE 1=1"); - statement.close(); - connection.close(); - } - - private void testJDBCSelect() throws Exception { - clearData(); - // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + } TupleStream stream; List tuples; // Simple 1 - stream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by CODE", new FieldComparator("CODE", ComparatorOrder.ASCENDING)); + stream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by CODE", + new FieldComparator("CODE", ComparatorOrder.ASCENDING)); tuples = getTuples(stream); assert(tuples.size() == 4); @@ -191,7 +137,8 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "COUNTRY_NAME", "Netherlands", "Norway", "Nepal", "United States"); // Simple 2 - stream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by COUNTRY_NAME", new FieldComparator("COUNTRY_NAME", ComparatorOrder.ASCENDING)); + stream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by COUNTRY_NAME", + new FieldComparator("COUNTRY_NAME", ComparatorOrder.ASCENDING)); tuples = getTuples(stream); assertEquals(4, tuples.size()); @@ -199,29 +146,28 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "COUNTRY_NAME", "Nepal", "Netherlands", "Norway", "United States"); } - - private void testJDBCJoin() throws Exception { - clearData(); + + @Test + public void testJDBCJoin() throws Exception { // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','NI')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NG')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NF')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NE')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','NC')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NZ')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','NR')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','NI')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NG')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NF')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NE')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','NC')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NZ')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','NR')"); + } TupleStream stream; List tuples; @@ -234,28 +180,28 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "ID", 11, 17, 19); assertOrderOf(tuples, "NAME", "Emma", "Mia", "Olivia"); } - - private void testJDBCSolrMerge() throws Exception { - clearData(); + + @Test + public void testJDBCSolrMerge() throws Exception { // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('AL', 'Algeria')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('AL', 'Algeria')"); + } // Load Solr - indexr(id, "0", "code_s", "GB", "name_s", "Great Britian"); - indexr(id, "1", "code_s", "CA", "name_s", "Canada"); - commit(); + new UpdateRequest() + .add(id, "0", "code_s", "GB", "name_s", "Great Britian") + .add(id, "1", "code_s", "CA", "name_s", "Canada") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class); List tuples; @@ -263,7 +209,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { // Simple 1 TupleStream jdbcStream = new JDBCStream("jdbc:hsqldb:mem:.", "select CODE,COUNTRY_NAME from COUNTRIES order by CODE", new FieldComparator("CODE", ComparatorOrder.ASCENDING)); TupleStream selectStream = new SelectStream(jdbcStream, new HashMap(){{ put("CODE", "code_s"); put("COUNTRY_NAME", "name_s"); }}); - TupleStream searchStream = factory.constructStream("search(collection1, fl=\"code_s,name_s\",q=\"*:*\",sort=\"code_s asc\")"); + TupleStream searchStream = factory.constructStream("search(" + COLLECTION + ", fl=\"code_s,name_s\",q=\"*:*\",sort=\"code_s asc\")"); TupleStream mergeStream = new MergeStream(new FieldComparator("code_s", ComparatorOrder.ASCENDING), new TupleStream[]{selectStream,searchStream}); tuples = getTuples(mergeStream); @@ -272,49 +218,49 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "code_s", "AL","CA","GB","NL","NO","NP","US"); assertOrderOf(tuples, "name_s", "Algeria", "Canada", "Great Britian", "Netherlands", "Norway", "Nepal", "United States"); } - - private void testJDBCSolrInnerJoinExpression() throws Exception{ - clearData(); + + @Test + public void testJDBCSolrInnerJoinExpression() throws Exception{ StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("select", SelectStream.class) .withFunctionName("innerJoin", InnerJoinStream.class) .withFunctionName("jdbc", JDBCStream.class); // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); + } // Load solr data - indexr(id, "1", "rating_f", "3.5", "personId_i", "11"); - indexr(id, "2", "rating_f", "5", "personId_i", "12"); - indexr(id, "3", "rating_f", "2.2", "personId_i", "13"); - indexr(id, "4", "rating_f", "4.3", "personId_i", "14"); - indexr(id, "5", "rating_f", "3.5", "personId_i", "15"); - indexr(id, "6", "rating_f", "3", "personId_i", "16"); - indexr(id, "7", "rating_f", "3", "personId_i", "17"); - indexr(id, "8", "rating_f", "4", "personId_i", "18"); - indexr(id, "9", "rating_f", "4.1", "personId_i", "19"); - indexr(id, "10", "rating_f", "4.8", "personId_i", "20"); - commit(); + new UpdateRequest() + .add(id, "1", "rating_f", "3.5", "personId_i", "11") + .add(id, "2", "rating_f", "5", "personId_i", "12") + .add(id, "3", "rating_f", "2.2", "personId_i", "13") + .add(id, "4", "rating_f", "4.3", "personId_i", "14") + .add(id, "5", "rating_f", "3.5", "personId_i", "15") + .add(id, "6", "rating_f", "3", "personId_i", "16") + .add(id, "7", "rating_f", "3", "personId_i", "17") + .add(id, "8", "rating_f", "4", "personId_i", "18") + .add(id, "9", "rating_f", "4.1", "personId_i", "19") + .add(id, "10", "rating_f", "4.8", "personId_i", "20") + .commit(cluster.getSolrClient(), COLLECTION); String expression; TupleStream stream; @@ -324,7 +270,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { expression = "innerJoin(" + " select(" - + " search(collection1, fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + + " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + " personId_i as personId," + " rating_f as rating" + " )," @@ -347,48 +293,48 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "country", "Netherlands","United States","Netherlands","Netherlands","Netherlands","United States","United States","Netherlands","Netherlands","United States"); } - private void testJDBCSolrInnerJoinExpressionWithProperties() throws Exception{ - clearData(); + @Test + public void testJDBCSolrInnerJoinExpressionWithProperties() throws Exception{ StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("select", SelectStream.class) .withFunctionName("innerJoin", InnerJoinStream.class) .withFunctionName("jdbc", JDBCStream.class); // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); + } // Load solr data - indexr(id, "1", "rating_f", "3.5", "personId_i", "11"); - indexr(id, "2", "rating_f", "5", "personId_i", "12"); - indexr(id, "3", "rating_f", "2.2", "personId_i", "13"); - indexr(id, "4", "rating_f", "4.3", "personId_i", "14"); - indexr(id, "5", "rating_f", "3.5", "personId_i", "15"); - indexr(id, "6", "rating_f", "3", "personId_i", "16"); - indexr(id, "7", "rating_f", "3", "personId_i", "17"); - indexr(id, "8", "rating_f", "4", "personId_i", "18"); - indexr(id, "9", "rating_f", "4.1", "personId_i", "19"); - indexr(id, "10", "rating_f", "4.8", "personId_i", "20"); - commit(); + new UpdateRequest() + .add(id, "1", "rating_f", "3.5", "personId_i", "11") + .add(id, "2", "rating_f", "5", "personId_i", "12") + .add(id, "3", "rating_f", "2.2", "personId_i", "13") + .add(id, "4", "rating_f", "4.3", "personId_i", "14") + .add(id, "5", "rating_f", "3.5", "personId_i", "15") + .add(id, "6", "rating_f", "3", "personId_i", "16") + .add(id, "7", "rating_f", "3", "personId_i", "17") + .add(id, "8", "rating_f", "4", "personId_i", "18") + .add(id, "9", "rating_f", "4.1", "personId_i", "19") + .add(id, "10", "rating_f", "4.8", "personId_i", "20") + .commit(cluster.getSolrClient(), COLLECTION); String expression; TupleStream stream; @@ -401,7 +347,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { expression = "innerJoin(" + " select(" - + " search(collection1, fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + + " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + " personId_i as personId," + " rating_f as rating" + " )," @@ -430,7 +376,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { expression = "innerJoin(" + " select(" - + " search(collection1, fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + + " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + " personId_i as personId," + " rating_f as rating" + " )," @@ -453,12 +399,11 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { assertOrderOf(tuples, "country", "Netherlands","United States","Netherlands","Netherlands","Netherlands","United States","United States","Netherlands","Netherlands","United States"); } - - private void testJDBCSolrInnerJoinRollupExpression() throws Exception{ - clearData(); + @Test + public void testJDBCSolrInnerJoinRollupExpression() throws Exception{ StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("select", SelectStream.class) .withFunctionName("hashJoin", HashJoinStream.class) @@ -471,38 +416,37 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { ; // Load Database Data - Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); - Statement statement = connection.createStatement(); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); - statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); - statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); - statement.close(); - connection.close(); + try (Connection connection = DriverManager.getConnection("jdbc:hsqldb:mem:."); + Statement statement = connection.createStatement()) { + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('US', 'United States')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NL', 'Netherlands')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NP', 'Nepal')"); + statement.executeUpdate("insert into COUNTRIES (CODE,COUNTRY_NAME) values ('NO', 'Norway')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (11,'Emma','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (12,'Grace','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (13,'Hailey','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (14,'Isabella','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (15,'Lily','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (16,'Madison','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (17,'Mia','US')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (18,'Natalie','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (19,'Olivia','NL')"); + statement.executeUpdate("insert into PEOPLE (ID, NAME, COUNTRY_CODE) values (20,'Samantha','US')"); + } // Load solr data - indexr(id, "1", "rating_f", "3.5", "personId_i", "11"); - indexr(id, "3", "rating_f", "2.2", "personId_i", "13"); - indexr(id, "4", "rating_f", "4.3", "personId_i", "14"); - indexr(id, "5", "rating_f", "3.5", "personId_i", "15"); - indexr(id, "8", "rating_f", "4", "personId_i", "18"); - indexr(id, "9", "rating_f", "4.1", "personId_i", "19"); - - indexr(id, "2", "rating_f", "5", "personId_i", "12"); - indexr(id, "6", "rating_f", "3", "personId_i", "16"); - indexr(id, "7", "rating_f", "3", "personId_i", "17"); - indexr(id, "10", "rating_f", "4.8", "personId_i", "20"); - commit(); + new UpdateRequest() + .add(id, "1", "rating_f", "3.5", "personId_i", "11") + .add(id, "3", "rating_f", "2.2", "personId_i", "13") + .add(id, "4", "rating_f", "4.3", "personId_i", "14") + .add(id, "5", "rating_f", "3.5", "personId_i", "15") + .add(id, "8", "rating_f", "4", "personId_i", "18") + .add(id, "9", "rating_f", "4.1", "personId_i", "19") + .add(id, "2", "rating_f", "5", "personId_i", "12") + .add(id, "6", "rating_f", "3", "personId_i", "16") + .add(id, "7", "rating_f", "3", "personId_i", "17") + .add(id, "10", "rating_f", "4.8", "personId_i", "20") + .commit(cluster.getSolrClient(), COLLECTION); String expression; TupleStream stream; @@ -513,7 +457,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { "rollup(" + " hashJoin(" + " hashed=select(" - + " search(collection1, fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + + " search(" + COLLECTION + ", fl=\"personId_i,rating_f\", q=\"rating_f:*\", sort=\"personId_i asc\")," + " personId_i as personId," + " rating_f as rating" + " )," @@ -562,6 +506,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { tupleStream.close(); return tuples; } + protected boolean assertOrderOf(List tuples, String fieldName, int... values) throws Exception { int i = 0; for(int val : values) { @@ -574,6 +519,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { } return true; } + protected boolean assertOrderOf(List tuples, String fieldName, double... values) throws Exception { int i = 0; for(double val : values) { @@ -586,6 +532,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { } return true; } + protected boolean assertOrderOf(List tuples, String fieldName, String... values) throws Exception { int i = 0; for(String val : values) { @@ -617,6 +564,7 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { } return true; } + protected boolean assertNotFields(List tuples, String ... fields) throws Exception{ for(Tuple tuple : tuples){ for(String field : fields){ @@ -649,9 +597,4 @@ public class JDBCStreamTest extends AbstractFullDistribZkTestBase { return true; } - @Override - protected void indexr(Object... fields) throws Exception { - SolrInputDocument doc = getDoc(fields); - indexDoc(doc); - } } diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java index 868afd5a2fa..9a0653a3930 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java @@ -16,41 +16,37 @@ */ package org.apache.solr.client.solrj.io.stream; -import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Collections; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.io.SolrClientCache; import org.apache.solr.client.solrj.io.Tuple; -import org.apache.solr.client.solrj.io.ops.ConcatOperation; -import org.apache.solr.client.solrj.io.ops.GroupOperation; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; import org.apache.solr.client.solrj.io.comp.FieldComparator; +import org.apache.solr.client.solrj.io.ops.ConcatOperation; +import org.apache.solr.client.solrj.io.ops.GroupOperation; import org.apache.solr.client.solrj.io.ops.ReplaceOperation; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -import org.apache.solr.client.solrj.io.stream.metrics.Bucket; import org.apache.solr.client.solrj.io.stream.metrics.CountMetric; import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric; import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric; -import org.apache.solr.client.solrj.io.stream.metrics.Metric; import org.apache.solr.client.solrj.io.stream.metrics.MinMetric; import org.apache.solr.client.solrj.io.stream.metrics.SumMetric; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.params.CommonParams; -import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -63,122 +59,50 @@ import org.junit.Test; @Slow @LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"}) -public class StreamExpressionTest extends AbstractFullDistribZkTestBase { +public class StreamExpressionTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); + private static final String COLLECTION = "collection1"; - static { - schemaString = "schema-streaming.xml"; - } + private static final int TIMEOUT = 30; + + private static final String id = "id"; @BeforeClass - public static void beforeSuperClass() { - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME()); - } + public static void setupCluster() throws Exception { + configureCluster(4) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); - @AfterClass - public static void afterSuperClass() { - - } - - protected String getCloudSolrConfig() { - return "solrconfig-streaming.xml"; - } - - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - public static String SOLR_HOME() { - return SOLR_HOME; + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); } @Before - @Override - public void setUp() throws Exception { - super.setUp(); - // we expect this time of exception as shards go up and down... - //ignoreException(".*"); - - System.setProperty("numShards", Integer.toString(sliceCount)); - } - - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - resetExceptionIgnores(); - } - - public StreamExpressionTest() { - super(); - sliceCount = 2; + public void cleanIndex() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); } @Test - public void testAll() throws Exception{ - assertNotNull(cloudClient); + public void testCloudSolrStream() throws Exception { - handle.clear(); - handle.put("timestamp", SKIPVAL); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); - waitForRecoveriesToFinish(false); - - del("*:*"); - commit(); - - testCloudSolrStream(); - testCloudSolrStreamWithZkHost(); - testMergeStream(); - testRankStream(); - testReducerStream(); - testUniqueStream(); - testSortStream(); - testRollupStream(); - testStatsStream(); - testNulls(); - testTopicStream(); - testDaemonStream(); - testRandomStream(); - testParallelUniqueStream(); - testParallelReducerStream(); - testParallelRankStream(); - testParallelMergeStream(); - testParallelRollupStream(); - testInnerJoinStream(); - testLeftOuterJoinStream(); - testHashJoinStream(); - testOuterHashJoinStream(); - testSelectStream(); - testFacetStream(); - testSubFacetStream(); - testUpdateStream(); - testParallelUpdateStream(); - testParallelDaemonUpdateStream(); - testIntersectStream(); - testParallelIntersectStream(); - testComplementStream(); - testParallelComplementStream(); - } - - private void testCloudSolrStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); - - StreamFactory factory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress()); + StreamFactory factory = new StreamFactory().withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()); StreamExpression expression; CloudSolrStream stream; List tuples; // Basic test - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -187,7 +111,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertLong(tuples.get(0), "a_i", 0); // Basic w/aliases - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -197,26 +121,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertString(tuples.get(0), "name", "hello0"); // Basic filtered test - expression = StreamExpressionParser.parse("search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 3); assertOrder(tuples, 0, 3, 4); assertLong(tuples.get(1), "a_i", 3); - - del("*:*"); - commit(); + } - private void testCloudSolrStreamWithZkHost() throws Exception { + @Test + public void testCloudSolrStreamWithZkHost() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory(); StreamExpression expression; @@ -224,7 +148,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { List tuples; // Basic test - expression = StreamExpressionParser.parse("search(collection1, zkHost=" + zkServer.getZkAddress() + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", zkHost=" + cluster.getZkServer().getZkAddress() + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -233,7 +157,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertLong(tuples.get(0), "a_i", 0); // Basic w/aliases - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\", zkHost=" + zkServer.getZkAddress() + ")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", aliases=\"a_i=alias.a_i, a_s=name\", zkHost=" + cluster.getZkServer().getZkAddress() + ")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -243,39 +167,39 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertString(tuples.get(0), "name", "hello0"); // Basic filtered test - expression = StreamExpressionParser.parse("search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", zkHost=" + zkServer.getZkAddress() + ", sort=\"a_f asc, a_i asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", zkHost=" + + cluster.getZkServer().getZkAddress() + ", sort=\"a_f asc, a_i asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 3); assertOrder(tuples, 0, 3, 4); assertLong(tuples.get(1), "a_i", 3); - - del("*:*"); - commit(); + } - - private void testUniqueStream() throws Exception { + @Test + public void testUniqueStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class); // Basic test - expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"); + expression = StreamExpressionParser.parse("unique(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f\")"); stream = new UniqueStream(expression, factory); tuples = getTuples(stream); @@ -283,7 +207,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertOrder(tuples, 0, 1, 3, 4); // Basic test desc - expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc, a_i desc\"), over=\"a_f\")"); + expression = StreamExpressionParser.parse("unique(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc, a_i desc\"), over=\"a_f\")"); stream = new UniqueStream(expression, factory); tuples = getTuples(stream); @@ -291,7 +215,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertOrder(tuples, 4,3,1,2); // Basic w/multi comp - expression = StreamExpressionParser.parse("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f, a_i\")"); + expression = StreamExpressionParser.parse("unique(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f, a_i\")"); stream = new UniqueStream(expression, factory); tuples = getTuples(stream); @@ -299,76 +223,75 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertOrder(tuples, 0,2,1,3,4); // full factory w/multi comp - stream = factory.constructStream("unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f, a_i\")"); + stream = factory.constructStream("unique(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\"), over=\"a_f, a_i\")"); tuples = getTuples(stream); assert(tuples.size() == 5); assertOrder(tuples, 0, 2, 1, 3, 4); - - del("*:*"); - commit(); + } - private void testSortStream() throws Exception { + @Test + public void testSortStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello1", "a_i", "1", "a_f", "2") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("sort", SortStream.class); // Basic test - stream = factory.constructStream("sort(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")"); + stream = factory.constructStream("sort(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")"); tuples = getTuples(stream); assert(tuples.size() == 6); assertOrder(tuples, 0,1,5,2,3,4); // Basic test desc - stream = factory.constructStream("sort(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i desc\")"); + stream = factory.constructStream("sort(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i desc\")"); tuples = getTuples(stream); assert(tuples.size() == 6); assertOrder(tuples, 4,3,2,1,5,0); // Basic w/multi comp - stream = factory.constructStream("sort(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc, a_f desc\")"); + stream = factory.constructStream("sort(search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc, a_f desc\")"); tuples = getTuples(stream); assert(tuples.size() == 6); assertOrder(tuples, 0,5,1,2,3,4); - - del("*:*"); - commit(); + } + @Test + public void testNulls() throws Exception { - private void testNulls() throws Exception { - - indexr(id, "0", "a_i", "1", "a_f", "0", "s_multi", "aaa", "s_multi", "bbb", "i_multi", "100", "i_multi", "200"); - indexr(id, "2", "a_s", "hello2", "a_i", "3", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "4", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "2", "a_f", "1"); - commit(); + new UpdateRequest() + .add(id, "0", "a_i", "1", "a_f", "0", "s_multi", "aaa", "s_multi", "bbb", "i_multi", "100", "i_multi", "200") + .add(id, "2", "a_s", "hello2", "a_i", "3", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "4", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "2", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; Tuple tuple; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class); // Basic test - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_i asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_i asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -392,7 +315,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertNotNull(longs); //test sort (asc) with null string field. Null should sort to the top. - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_s asc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_s asc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); @@ -400,41 +323,40 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertOrder(tuples, 0, 1, 2, 3, 4); //test sort(desc) with null string field. Null should sort to the bottom. - expression = StreamExpressionParser.parse("search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_s desc\")"); + expression = StreamExpressionParser.parse("search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f, s_multi, i_multi\", qt=\"/export\", sort=\"a_s desc\")"); stream = new CloudSolrStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 5); assertOrder(tuples, 4, 3, 2, 1, 0); - del("*:*"); - commit(); } - - private void testMergeStream() throws Exception { + @Test + public void testMergeStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class) .withFunctionName("merge", MergeStream.class); // Basic test expression = StreamExpressionParser.parse("merge(" - + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")," - + "search(collection1, q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")," + + "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")," + + "search(" + COLLECTION + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\")," + "on=\"a_f asc\")"); stream = new MergeStream(expression, factory); tuples = getTuples(stream); @@ -444,8 +366,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Basic test desc expression = StreamExpressionParser.parse("merge(" - + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," - + "search(collection1, q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," + + "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," + + "search(" + COLLECTION + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," + "on=\"a_f desc\")"); stream = new MergeStream(expression, factory); tuples = getTuples(stream); @@ -455,8 +377,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Basic w/multi comp expression = StreamExpressionParser.parse("merge(" - + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," - + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + "on=\"a_f asc, a_s asc\")"); stream = new MergeStream(expression, factory); tuples = getTuples(stream); @@ -466,8 +388,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // full factory w/multi comp stream = factory.constructStream("merge(" - + "search(collection1, q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," - + "search(collection1, q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + "on=\"a_f asc, a_s asc\")"); tuples = getTuples(stream); @@ -476,34 +398,34 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // full factory w/multi streams stream = factory.constructStream("merge(" - + "search(collection1, q=\"id:(0 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," - + "search(collection1, q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," - + "search(collection1, q=\"id:(2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(0 4)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + + "search(" + COLLECTION + ", q=\"id:(2)\", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_s asc\")," + "on=\"a_f asc\")"); tuples = getTuples(stream); assert(tuples.size() == 4); assertOrder(tuples, 0, 2, 1, 4); - - del("*:*"); - commit(); - } - - private void testRankStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); + } + + @Test + public void testRankStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class) .withFunctionName("top", RankStream.class); @@ -511,7 +433,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Basic test expression = StreamExpressionParser.parse("top(" + "n=3," - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")," + "sort=\"a_f asc, a_i asc\")"); stream = new RankStream(expression, factory); tuples = getTuples(stream); @@ -523,7 +445,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { expression = StreamExpressionParser.parse("top(" + "n=2," + "unique(" - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc\")," + "over=\"a_f\")," + "sort=\"a_f desc\")"); stream = new RankStream(expression, factory); @@ -536,7 +458,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { stream = factory.constructStream("top(" + "n=4," + "unique(" - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")," + "over=\"a_f\")," + "sort=\"a_f asc\")"); tuples = getTuples(stream); @@ -548,32 +470,32 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { stream = factory.constructStream("top(" + "n=4," + "unique(" - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc, a_i desc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f desc, a_i desc\")," + "over=\"a_f\")," + "sort=\"a_f asc\")"); tuples = getTuples(stream); assert(tuples.size() == 4); assertOrder(tuples, 2,1,3,4); - - del("*:*"); - commit(); + } - private void testRandomStream() throws Exception { + @Test + public void testRandomStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("random", RandomStream.class); @@ -582,13 +504,13 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { try { context.setSolrClientCache(cache); - expression = StreamExpressionParser.parse("random(collection1, q=\"*:*\", rows=\"10\", fl=\"id, a_i\")"); + expression = StreamExpressionParser.parse("random(" + COLLECTION + ", q=\"*:*\", rows=\"10\", fl=\"id, a_i\")"); stream = factory.constructStream(expression); stream.setStreamContext(context); List tuples1 = getTuples(stream); assert (tuples1.size() == 5); - expression = StreamExpressionParser.parse("random(collection1, q=\"*:*\", rows=\"10\", fl=\"id, a_i\")"); + expression = StreamExpressionParser.parse("random(" + COLLECTION + ", q=\"*:*\", rows=\"10\", fl=\"id, a_i\")"); stream = factory.constructStream(expression); stream.setStreamContext(context); List tuples2 = getTuples(stream); @@ -617,7 +539,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { } } - expression = StreamExpressionParser.parse("random(collection1, q=\"*:*\", rows=\"1\", fl=\"id, a_i\")"); + expression = StreamExpressionParser.parse("random(" + COLLECTION + ", q=\"*:*\", rows=\"1\", fl=\"id, a_i\")"); stream = factory.constructStream(expression); stream.setStreamContext(context); List tuples3 = getTuples(stream); @@ -625,23 +547,24 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { } finally { cache.close(); - del("*:*"); - commit(); } } - - private void testReducerStream() throws Exception{ - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - commit(); + + @Test + public void testReducerStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; @@ -650,14 +573,14 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { List maps0, maps1, maps2; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("reduce", ReducerStream.class) .withFunctionName("group", GroupOperation.class); // basic expression = StreamExpressionParser.parse("reduce(" - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\")," + "by=\"a_s\"," + "group(sort=\"a_f desc\", n=\"4\"))"); @@ -681,7 +604,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // basic w/spaces expression = StreamExpressionParser.parse("reduce(" - + "search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc, a_f asc\")," + "by=\"a_s\"," + "group(sort=\"a_i asc\", n=\"2\"))"); stream = factory.constructStream(expression); @@ -703,27 +626,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { maps2 = t2.getMaps("group"); assertMaps(maps2, 4, 6); - del("*:*"); - commit(); } - private void testDaemonStream() throws Exception { + @Test + public void testDaemonStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("rollup", RollupStream.class) .withFunctionName("sum", SumMetric.class) @@ -737,7 +659,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { DaemonStream daemonStream; expression = StreamExpressionParser.parse("daemon(rollup(" - + "search(collection1, q=\"*:*\", fl=\"a_i,a_s\", sort=\"a_s asc\")," + + "search(" + COLLECTION + ", q=\"*:*\", fl=\"a_i,a_s\", sort=\"a_s asc\")," + "over=\"a_s\"," + "sum(a_i)" + "), id=\"test\", runInterval=\"1000\", queueSize=\"9\")"); @@ -785,8 +707,9 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { //OK capacity is full, let's index a new doc - indexr(id, "10", "a_s", "hello0", "a_i", "1", "a_f", "10"); - commit(); + new UpdateRequest() + .add(id, "10", "a_s", "hello0", "a_i", "1", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Now lets clear the existing docs in the queue 9, plus 3 more to get passed the run that was blocked. The next run should //have the tuples with the updated count. @@ -822,32 +745,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { daemonStream.close(); //This should stop the daemon thread - del("*:*"); - commit(); } + @Test + public void testRollupStream() throws Exception { - - - - - private void testRollupStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("rollup", RollupStream.class) .withFunctionName("sum", SumMetric.class) @@ -861,7 +778,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { List tuples; expression = StreamExpressionParser.parse("rollup(" - + "search(collection1, q=*:*, fl=\"a_s,a_i,a_f\", sort=\"a_s asc\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"a_s,a_i,a_f\", sort=\"a_s asc\")," + "over=\"a_s\"," + "sum(a_i)," + "sum(a_f)," @@ -949,27 +866,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } - - private void testStatsStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); + @Test + public void testStatsStream() throws Exception { - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("stats", StatsStream.class) .withFunctionName("sum", SumMetric.class) .withFunctionName("min", MinMetric.class) @@ -1012,33 +928,32 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 10); - del("*:*"); - commit(); } - - private void testParallelUniqueStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello1", "a_i", "10", "a_f", "1"); - indexr(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5"); - indexr(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5"); - indexr(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4"); + @Test + public void testParallelUniqueStream() throws Exception { - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello1", "a_i", "10", "a_f", "1") + .add(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5") + .add(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5") + .add(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4") + .commit(cluster.getSolrClient(), COLLECTION); - String zkHost = zkServer.getZkAddress(); - StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress()) + String zkHost = cluster.getZkServer().getZkAddress(); + StreamFactory streamFactory = new StreamFactory().withCollectionZkHost(COLLECTION, zkHost) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class) .withFunctionName("top", RankStream.class) .withFunctionName("group", ReducerStream.class) .withFunctionName("parallel", ParallelStream.class); - ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", partitionKeys=\"a_f\"), over=\"a_f\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_f asc\")"); + ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(" + COLLECTION + ", unique(search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\", partitionKeys=\"a_f\"), over=\"a_f\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_f asc\")"); List tuples = getTuples(pstream); assert(tuples.size() == 5); @@ -1049,36 +964,34 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { Map eofTuples = pstream.getEofTuples(); assert(eofTuples.size() == 2); //There should be an EOF tuple for each worker. - del("*:*"); - commit(); - } - private void testParallelReducerStream() throws Exception { + @Test + public void testParallelReducerStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); - commit(); - - String zkHost = zkServer.getZkAddress(); - StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress()) + String zkHost = cluster.getZkServer().getZkAddress(); + StreamFactory streamFactory = new StreamFactory().withCollectionZkHost(COLLECTION, zkHost) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("group", GroupOperation.class) .withFunctionName("reduce", ReducerStream.class) .withFunctionName("parallel", ParallelStream.class); - ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1," + + ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(" + COLLECTION + ", " + "reduce(" + - "search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc,a_f asc\", partitionKeys=\"a_s\"), " + + "search(" + COLLECTION + ", q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc,a_f asc\", partitionKeys=\"a_s\"), " + "by=\"a_s\"," + "group(sort=\"a_i asc\", n=\"5\")), " + "workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_s asc\")"); @@ -1100,9 +1013,9 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertMaps(maps2, 4, 6); - pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, " + + pstream = (ParallelStream)streamFactory.constructStream("parallel(" + COLLECTION + ", " + "reduce(" + - "search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc,a_f asc\", partitionKeys=\"a_s\"), " + + "search(" + COLLECTION + ", q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s desc,a_f asc\", partitionKeys=\"a_s\"), " + "by=\"a_s\", " + "group(sort=\"a_i desc\", n=\"5\")),"+ "workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_s desc\")"); @@ -1125,27 +1038,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { maps2 = t2.getMaps("group"); assertMaps(maps2, 9, 2, 1, 0); - del("*:*"); - commit(); } - private void testParallelRankStream() throws Exception { + @Test + public void testParallelRankStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "5", "a_s", "hello1", "a_i", "5", "a_f", "1"); - indexr(id, "6", "a_s", "hello1", "a_i", "6", "a_f", "1"); - indexr(id, "7", "a_s", "hello1", "a_i", "7", "a_f", "1"); - indexr(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1"); - indexr(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1"); - indexr(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1"); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "5", "a_s", "hello1", "a_i", "5", "a_f", "1") + .add(id, "6", "a_s", "hello1", "a_i", "6", "a_f", "1") + .add(id, "7", "a_s", "hello1", "a_i", "7", "a_f", "1") + .add(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1") + .add(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1") + .add(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); - commit(); - - String zkHost = zkServer.getZkAddress(); - StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress()) + String zkHost = cluster.getZkServer().getZkAddress(); + StreamFactory streamFactory = new StreamFactory().withCollectionZkHost(COLLECTION, zkHost) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class) .withFunctionName("top", RankStream.class) @@ -1153,9 +1065,9 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { .withFunctionName("parallel", ParallelStream.class); ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(" - + "collection1, " + + COLLECTION + ", " + "top(" - + "search(collection1, q=\"*:*\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), " + + "search(" + COLLECTION + ", q=\"*:*\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), " + "n=\"11\", " + "sort=\"a_i desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i desc\")"); @@ -1164,27 +1076,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 10); assertOrder(tuples, 10,9,8,7,6,5,4,3,2,0); - del("*:*"); - commit(); } - private void testParallelMergeStream() throws Exception { + @Test + public void testParallelMergeStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0"); - indexr(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0"); - indexr(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3"); - indexr(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4"); - indexr(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1"); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0") + .add(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0") + .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3") + .add(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4") + .add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); - commit(); - - String zkHost = zkServer.getZkAddress(); - StreamFactory streamFactory = new StreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress()) + String zkHost = cluster.getZkServer().getZkAddress(); + StreamFactory streamFactory = new StreamFactory().withCollectionZkHost(COLLECTION, zkHost) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("unique", UniqueStream.class) .withFunctionName("top", RankStream.class) @@ -1193,7 +1104,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { .withFunctionName("parallel", ParallelStream.class); //Test ascending - ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, merge(search(collection1, q=\"id:(4 1 8 7 9)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), search(collection1, q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), on=\"a_i asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i asc\")"); + ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(" + COLLECTION + ", merge(search(" + COLLECTION + ", q=\"id:(4 1 8 7 9)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), search(" + COLLECTION + ", q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i asc\", partitionKeys=\"a_i\"), on=\"a_i asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i asc\")"); List tuples = getTuples(pstream); @@ -1204,34 +1115,33 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { //Test descending - pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, merge(search(collection1, q=\"id:(4 1 8 9)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), search(collection1, q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), on=\"a_i desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i desc\")"); + pstream = (ParallelStream)streamFactory.constructStream("parallel(" + COLLECTION + ", merge(search(" + COLLECTION + ", q=\"id:(4 1 8 9)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), search(" + COLLECTION + ", q=\"id:(0 2 3 6)\", fl=\"id,a_s,a_i\", sort=\"a_i desc\", partitionKeys=\"a_i\"), on=\"a_i desc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_i desc\")"); tuples = getTuples(pstream); assert(tuples.size() == 8); assertOrder(tuples, 9,8,6,4,3,2,1,0); - del("*:*"); - commit(); } - - private void testParallelRollupStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); + @Test + public void testParallelRollupStream() throws Exception { - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("parallel", ParallelStream.class) .withFunctionName("rollup", RollupStream.class) @@ -1245,9 +1155,9 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { TupleStream stream; List tuples; - expression = StreamExpressionParser.parse("parallel(collection1," + expression = StreamExpressionParser.parse("parallel(" + COLLECTION + "," + "rollup(" - + "search(collection1, q=*:*, fl=\"a_s,a_i,a_f\", sort=\"a_s asc\", partitionKeys=\"a_s\")," + + "search(" + COLLECTION + ", q=*:*, fl=\"a_s,a_i,a_f\", sort=\"a_s asc\", partitionKeys=\"a_s\")," + "over=\"a_s\"," + "sum(a_i)," + "sum(a_f)," @@ -1259,7 +1169,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { + "avg(a_f)," + "count(*)" + ")," - + "workers=\"2\", zkHost=\""+zkServer.getZkAddress()+"\", sort=\"a_s asc\")" + + "workers=\"2\", zkHost=\""+cluster.getZkServer().getZkAddress()+"\", sort=\"a_s asc\")" ); stream = factory.constructStream(expression); tuples = getTuples(stream); @@ -1337,43 +1247,43 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } - private void testInnerJoinStream() throws Exception { + @Test + public void testInnerJoinStream() throws Exception { - indexr(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2"); - indexr(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3"); // 10 - indexr(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4"); // 11 - indexr(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5"); // 12 - indexr(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6"); - indexr(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7"); // 14 + new UpdateRequest() + .add(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2") + .add(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3") // 10 + .add(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4") // 11 + .add(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5") // 12 + .add(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6") + .add(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7") // 14 - indexr(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0"); // 1,15 - indexr(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0"); // 1,15 - indexr(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1"); // 3 - indexr(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1"); // 4 - indexr(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1"); // 5 - indexr(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2"); - indexr(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3"); // 7 - commit(); + .add(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0") // 1,15 + .add(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0") // 1,15 + .add(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1") // 3 + .add(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1") // 4 + .add(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1") // 5 + .add(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2") + .add(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3") // 7 + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("innerJoin", InnerJoinStream.class); // Basic test expression = StreamExpressionParser.parse("innerJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc\")," + "on=\"join1_i=join1_i, join2_s=join2_s\")"); stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); @@ -1382,8 +1292,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Basic desc expression = StreamExpressionParser.parse("innerJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + "on=\"join1_i=join1_i, join2_s=join2_s\")"); stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); @@ -1392,8 +1302,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Results in both searches, no join matches expression = StreamExpressionParser.parse("innerJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," + "on=\"ident_s=right.ident_s\")"); stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); @@ -1401,52 +1311,52 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Differing field names expression = StreamExpressionParser.parse("innerJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join3_i,join2_s,ident_s\", sort=\"join3_i asc, join2_s asc\", aliases=\"join3_i=aliasesField\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join3_i,join2_s,ident_s\", sort=\"join3_i asc, join2_s asc\", aliases=\"join3_i=aliasesField\")," + "on=\"join1_i=aliasesField, join2_s=join2_s\")"); stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 8); assertOrder(tuples, 1,1,15,15,3,4,5,7); - - del("*:*"); - commit(); + } - - private void testLeftOuterJoinStream() throws Exception { - indexr(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2"); - indexr(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3"); // 10 - indexr(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4"); // 11 - indexr(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5"); // 12 - indexr(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6"); - indexr(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7"); // 14 + @Test + public void testLeftOuterJoinStream() throws Exception { - indexr(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0"); // 1,15 - indexr(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0"); // 1,15 - indexr(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1"); // 3 - indexr(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1"); // 4 - indexr(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1"); // 5 - indexr(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2"); - indexr(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3"); // 7 - commit(); + new UpdateRequest() + .add(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2") + .add(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3") // 10 + .add(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4") // 11 + .add(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5") // 12 + .add(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6") + .add(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7") // 14 + + .add(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0") // 1,15 + .add(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0") // 1,15 + .add(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1") // 3 + .add(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1") // 4 + .add(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1") // 5 + .add(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2") + .add(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3") // 7 + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("leftOuterJoin", LeftOuterJoinStream.class); // Basic test expression = StreamExpressionParser.parse("leftOuterJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc\")," + "on=\"join1_i=join1_i, join2_s=join2_s\")"); stream = new LeftOuterJoinStream(expression, factory); tuples = getTuples(stream); @@ -1455,8 +1365,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Basic desc expression = StreamExpressionParser.parse("leftOuterJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join1_i,join2_s,ident_s\", sort=\"join1_i desc, join2_s asc\")," + "on=\"join1_i=join1_i, join2_s=join2_s\")"); stream = new LeftOuterJoinStream(expression, factory); tuples = getTuples(stream); @@ -1465,8 +1375,8 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Results in both searches, no join matches expression = StreamExpressionParser.parse("leftOuterJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," + "on=\"ident_s=right.ident_s\")"); stream = new LeftOuterJoinStream(expression, factory); tuples = getTuples(stream); @@ -1475,44 +1385,44 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { // Differing field names expression = StreamExpressionParser.parse("leftOuterJoin(" - + "search(collection1, q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," - + "search(collection1, q=\"side_s:right\", fl=\"join3_i,join2_s,ident_s\", sort=\"join3_i asc, join2_s asc\", aliases=\"join3_i=aliasesField\")," + + "search(" + COLLECTION + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"join1_i asc, join2_s asc, id asc\")," + + "search(" + COLLECTION + ", q=\"side_s:right\", fl=\"join3_i,join2_s,ident_s\", sort=\"join3_i asc, join2_s asc\", aliases=\"join3_i=aliasesField\")," + "on=\"join1_i=aliasesField, join2_s=join2_s\")"); stream = new LeftOuterJoinStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 10); assertOrder(tuples, 1,1,15,15,2,3,4,5,6,7); - - del("*:*"); - commit(); + } - private void testHashJoinStream() throws Exception { + @Test + public void testHashJoinStream() throws Exception { - indexr(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2"); - indexr(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3"); // 10 - indexr(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4"); // 11 - indexr(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5"); // 12 - indexr(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6"); - indexr(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7"); // 14 + new UpdateRequest() + .add(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2") + .add(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3") // 10 + .add(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4") // 11 + .add(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5") // 12 + .add(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6") + .add(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7") // 14 - indexr(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0"); // 1,15 - indexr(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0"); // 1,15 - indexr(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1"); // 3 - indexr(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1"); // 4 - indexr(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1"); // 5 - indexr(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2"); - indexr(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3"); // 7 - commit(); + .add(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0") // 1,15 + .add(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0") // 1,15 + .add(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1") // 3 + .add(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1") // 4 + .add(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1") // 5 + .add(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2") + .add(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3") // 7 + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost(COLLECTION, cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("hashJoin", HashJoinStream.class); @@ -1544,37 +1454,37 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { stream = new HashJoinStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 0); - - del("*:*"); - commit(); + } - - private void testOuterHashJoinStream() throws Exception { - indexr(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2"); - indexr(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3"); // 10 - indexr(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4"); // 11 - indexr(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5"); // 12 - indexr(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6"); - indexr(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7"); // 14 + @Test + public void testOuterHashJoinStream() throws Exception { - indexr(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0"); // 1,15 - indexr(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0"); // 1,15 - indexr(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1"); // 3 - indexr(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1"); // 4 - indexr(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1"); // 5 - indexr(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2"); - indexr(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3"); // 7 - commit(); + new UpdateRequest() + .add(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2") + .add(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3") // 10 + .add(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4") // 11 + .add(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5") // 12 + .add(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6") + .add(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7") // 14 + + .add(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0") // 1,15 + .add(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0") // 1,15 + .add(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1") // 3 + .add(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1") // 4 + .add(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1") // 5 + .add(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2") + .add(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3") // 7 + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("outerHashJoin", OuterHashJoinStream.class); @@ -1607,37 +1517,37 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { tuples = getTuples(stream); assert(tuples.size() == 8); assertOrder(tuples, 1,15,2,3,4,5,6,7); - - del("*:*"); - commit(); + } - - private void testSelectStream() throws Exception { - indexr(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1"); // 8, 9 - indexr(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2"); - indexr(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3"); // 10 - indexr(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4"); // 11 - indexr(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5"); // 12 - indexr(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6"); - indexr(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7"); // 14 + @Test + public void testSelectStream() throws Exception { - indexr(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0"); // 1,15 - indexr(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0"); // 1,15 - indexr(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1"); // 3 - indexr(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1"); // 4 - indexr(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1"); // 5 - indexr(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2"); - indexr(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3"); // 7 - commit(); + new UpdateRequest() + .add(id, "1", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "15", "side_s", "left", "join1_i", "0", "join2_s", "a", "ident_s", "left_1") // 8, 9 + .add(id, "2", "side_s", "left", "join1_i", "0", "join2_s", "b", "ident_s", "left_2") + .add(id, "3", "side_s", "left", "join1_i", "1", "join2_s", "a", "ident_s", "left_3") // 10 + .add(id, "4", "side_s", "left", "join1_i", "1", "join2_s", "b", "ident_s", "left_4") // 11 + .add(id, "5", "side_s", "left", "join1_i", "1", "join2_s", "c", "ident_s", "left_5") // 12 + .add(id, "6", "side_s", "left", "join1_i", "2", "join2_s", "d", "ident_s", "left_6") + .add(id, "7", "side_s", "left", "join1_i", "3", "join2_s", "e", "ident_s", "left_7") // 14 + + .add(id, "8", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_1", "join3_i", "0") // 1,15 + .add(id, "9", "side_s", "right", "join1_i", "0", "join2_s", "a", "ident_s", "right_2", "join3_i", "0") // 1,15 + .add(id, "10", "side_s", "right", "join1_i", "1", "join2_s", "a", "ident_s", "right_3", "join3_i", "1") // 3 + .add(id, "11", "side_s", "right", "join1_i", "1", "join2_s", "b", "ident_s", "right_4", "join3_i", "1") // 4 + .add(id, "12", "side_s", "right", "join1_i", "1", "join2_s", "c", "ident_s", "right_5", "join3_i", "1") // 5 + .add(id, "13", "side_s", "right", "join1_i", "2", "join2_s", "dad", "ident_s", "right_6", "join3_i", "2") + .add(id, "14", "side_s", "right", "join1_i", "3", "join2_s", "e", "ident_s", "right_7", "join3_i", "3") // 7 + .commit(cluster.getSolrClient(), COLLECTION); String clause; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("innerJoin", InnerJoinStream.class) .withFunctionName("select", SelectStream.class) @@ -1729,32 +1639,31 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { tuples = getTuples(stream); assertFields(tuples, "id", "left.ident", "right.ident"); assertNotFields(tuples, "left.join1", "left.join2", "right.join1", "right.join2"); - - del("*:*"); - commit(); + } - - private void testFacetStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); + @Test + public void testFacetStream() throws Exception { - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); String clause; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("facet", FacetStream.class) .withFunctionName("sum", SumMetric.class) .withFunctionName("min", MinMetric.class) @@ -2136,32 +2045,30 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } + @Test + public void testSubFacetStream() throws Exception { - private void testSubFacetStream() throws Exception { - - indexr(id, "0", "level1_s", "hello0", "level2_s", "a", "a_i", "0", "a_f", "1"); - indexr(id, "2", "level1_s", "hello0", "level2_s", "a", "a_i", "2", "a_f", "2"); - indexr(id, "3", "level1_s", "hello3", "level2_s", "a", "a_i", "3", "a_f", "3"); - indexr(id, "4", "level1_s", "hello4", "level2_s", "a", "a_i", "4", "a_f", "4"); - indexr(id, "1", "level1_s", "hello0", "level2_s", "b", "a_i", "1", "a_f", "5"); - indexr(id, "5", "level1_s", "hello3", "level2_s", "b", "a_i", "10", "a_f", "6"); - indexr(id, "6", "level1_s", "hello4", "level2_s", "b", "a_i", "11", "a_f", "7"); - indexr(id, "7", "level1_s", "hello3", "level2_s", "b", "a_i", "12", "a_f", "8"); - indexr(id, "8", "level1_s", "hello3", "level2_s", "b", "a_i", "13", "a_f", "9"); - indexr(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10"); - - commit(); + new UpdateRequest() + .add(id, "0", "level1_s", "hello0", "level2_s", "a", "a_i", "0", "a_f", "1") + .add(id, "2", "level1_s", "hello0", "level2_s", "a", "a_i", "2", "a_f", "2") + .add(id, "3", "level1_s", "hello3", "level2_s", "a", "a_i", "3", "a_f", "3") + .add(id, "4", "level1_s", "hello4", "level2_s", "a", "a_i", "4", "a_f", "4") + .add(id, "1", "level1_s", "hello0", "level2_s", "b", "a_i", "1", "a_f", "5") + .add(id, "5", "level1_s", "hello3", "level2_s", "b", "a_i", "10", "a_f", "6") + .add(id, "6", "level1_s", "hello4", "level2_s", "b", "a_i", "11", "a_f", "7") + .add(id, "7", "level1_s", "hello3", "level2_s", "b", "a_i", "12", "a_f", "8") + .add(id, "8", "level1_s", "hello3", "level2_s", "b", "a_i", "13", "a_f", "9") + .add(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); String clause; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("facet", FacetStream.class) .withFunctionName("sum", SumMetric.class) .withFunctionName("min", MinMetric.class) @@ -2330,26 +2237,26 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertTrue(sumi.longValue() == 2); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } - private void testTopicStream() throws Exception{ - indexr(id, "0", "a_s", "hello", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello", "a_i", "14", "a_f", "10"); + @Test + public void testTopicStream() throws Exception { - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "hello", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("topic", TopicStream.class) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("daemon", DaemonStream.class); @@ -2373,7 +2280,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { //Should be zero because the checkpoints will be set to the highest vesion on the shards. assertEquals(tuples.size(), 0); - commit(); + cluster.getSolrClient().commit("collection1"); //Now check to see if the checkpoints are present expression = StreamExpressionParser.parse("search(collection1, q=\"id:1000000\", fl=\"id, checkpoint_ss, _version_\", sort=\"id asc\")"); @@ -2388,10 +2295,10 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { Long version1 = tuples.get(0).getLong("_version_"); //Index a few more documents - indexr(id, "10", "a_s", "hello", "a_i", "13", "a_f", "9"); - indexr(id, "11", "a_s", "hello", "a_i", "14", "a_f", "10"); - - commit(); + new UpdateRequest() + .add(id, "10", "a_s", "hello", "a_i", "13", "a_f", "9") + .add(id, "11", "a_s", "hello", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); expression = StreamExpressionParser.parse("topic(collection1, collection1, fl=\"id\", q=\"a_s:hello\", id=\"1000000\", checkpointEvery=2)"); @@ -2404,7 +2311,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { stream.open(); Tuple tuple1 = stream.read(); assertEquals((long) tuple1.getLong("id"), 10l); - commit(); + cluster.getSolrClient().commit("collection1"); // Checkpoint should not have changed. expression = StreamExpressionParser.parse("search(collection1, q=\"id:1000000\", fl=\"id, checkpoint_ss, _version_\", sort=\"id asc\")"); @@ -2421,7 +2328,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertEquals(version1, version2); Tuple tuple2 = stream.read(); - commit(); + cluster.getSolrClient().commit("collection1"); assertEquals((long) tuple2.getLong("id"), 11l); //Checkpoint should have changed. @@ -2455,9 +2362,10 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { dstream.setStreamContext(context); //Index a few more documents - indexr(id, "12", "a_s", "hello", "a_i", "13", "a_f", "9"); - indexr(id, "13", "a_s", "hello", "a_i", "14", "a_f", "10"); - commit(); + new UpdateRequest() + .add(id, "12", "a_s", "hello", "a_i", "13", "a_f", "9") + .add(id, "13", "a_s", "hello", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Start reading from the DaemonStream Tuple tuple = null; @@ -2467,12 +2375,13 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertEquals(12, (long) tuple.getLong(id)); tuple = dstream.read(); assertEquals(13, (long) tuple.getLong(id)); - commit(); // We want to see if the version has been updated after reading two tuples + cluster.getSolrClient().commit("collection1"); // We want to see if the version has been updated after reading two tuples //Index a few more documents - indexr(id, "14", "a_s", "hello", "a_i", "13", "a_f", "9"); - indexr(id, "15", "a_s", "hello", "a_i", "14", "a_f", "10"); - commit(); + new UpdateRequest() + .add(id, "14", "a_s", "hello", "a_i", "13", "a_f", "9") + .add(id, "15", "a_s", "hello", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Read from the same DaemonStream stream @@ -2488,31 +2397,32 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { dstream.close(); } } finally { - del("*:*"); - commit(); cache.close(); } } - private void testUpdateStream() throws Exception { - CloudSolrClient destinationCollectionClient = createCloudClient("destinationCollection"); - createCollection("destinationCollection", destinationCollectionClient, 2, 2); - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777"); - commit(); - waitForRecoveriesToFinish("destinationCollection", false); + @Test + public void testUpdateStream() throws Exception { + + CollectionAdminRequest.createCollection("destinationCollection", "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("destinationCollection", cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); + + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777") + .commit(cluster.getSolrClient(), "collection1"); StreamExpression expression; TupleStream stream; Tuple t; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) - .withCollectionZkHost("destinationCollection", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) + .withCollectionZkHost("destinationCollection", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("update", UpdateStream.class); @@ -2520,7 +2430,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { expression = StreamExpressionParser.parse("update(destinationCollection, batchSize=5, search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f,s_multi,i_multi\", sort=\"a_f asc, a_i asc\"))"); stream = new UpdateStream(expression, factory); List tuples = getTuples(stream); - destinationCollectionClient.commit(); + cluster.getSolrClient().commit("destinationCollection"); //Ensure that all UpdateStream tuples indicate the correct number of copied/indexed docs assert(tuples.size() == 1); @@ -2573,34 +2483,31 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("a_f") == 4.0); assertList(tuple.getStrings("s_multi"), "aaaa3", "bbbb3"); assertList(tuple.getLongs("i_multi"), Long.parseLong("4444"), Long.parseLong("7777")); - - destinationCollectionClient.deleteByQuery("*:*"); - destinationCollectionClient.commit(); - destinationCollectionClient.close(); - del("*:*"); - commit(); } - - private void testParallelUpdateStream() throws Exception { - CloudSolrClient destinationCollectionClient = createCloudClient("parallelDestinationCollection"); - createCollection("parallelDestinationCollection", destinationCollectionClient, 2, 2); - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777"); - commit(); - waitForRecoveriesToFinish("parallelDestinationCollection", false); + @Test + public void testParallelUpdateStream() throws Exception { + + CollectionAdminRequest.createCollection("parallelDestinationCollection", "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("parallelDestinationCollection", cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); + + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777") + .commit(cluster.getSolrClient(), "collection1"); StreamExpression expression; TupleStream stream; Tuple t; - String zkHost = zkServer.getZkAddress(); + String zkHost = cluster.getZkServer().getZkAddress(); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) - .withCollectionZkHost("parallelDestinationCollection", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) + .withCollectionZkHost("parallelDestinationCollection", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("update", UpdateStream.class) .withFunctionName("parallel", ParallelStream.class); @@ -2609,7 +2516,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { String updateExpression = "update(parallelDestinationCollection, batchSize=2, search(collection1, q=*:*, fl=\"id,a_s,a_i,a_f,s_multi,i_multi\", sort=\"a_f asc, a_i asc\", partitionKeys=\"a_f\"))"; TupleStream parallelUpdateStream = factory.constructStream("parallel(collection1, " + updateExpression + ", workers=\"2\", zkHost=\""+zkHost+"\", sort=\"batchNumber asc\")"); List tuples = getTuples(parallelUpdateStream); - destinationCollectionClient.commit(); + cluster.getSolrClient().commit("parallelDestinationCollection"); //Ensure that all UpdateStream tuples indicate the correct number of copied/indexed docs long count = 0; @@ -2666,33 +2573,31 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertList(tuple.getStrings("s_multi"), "aaaa3", "bbbb3"); assertList(tuple.getLongs("i_multi"), Long.parseLong("4444"), Long.parseLong("7777")); - destinationCollectionClient.deleteByQuery("*:*"); - destinationCollectionClient.commit(); - destinationCollectionClient.close(); - del("*:*"); - commit(); } - private void testParallelDaemonUpdateStream() throws Exception { - CloudSolrClient destinationCollectionClient = createCloudClient("parallelDestinationCollection1"); - createCollection("parallelDestinationCollection1", destinationCollectionClient, 2, 2); + @Test + public void testParallelDaemonUpdateStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777"); - commit(); - waitForRecoveriesToFinish("parallelDestinationCollection1", false); + CollectionAdminRequest.createCollection("parallelDestinationCollection1", "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish("parallelDestinationCollection1", cluster.getSolrClient().getZkStateReader(), + false, true, TIMEOUT); + + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1", "s_multi", "aaaa4", "s_multi", "bbbb4", "i_multi", "44444", "i_multi", "77777") + .commit(cluster.getSolrClient(), "collection1"); StreamExpression expression; TupleStream stream; Tuple t; - String zkHost = zkServer.getZkAddress(); + String zkHost = cluster.getZkServer().getZkAddress(); StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) - .withCollectionZkHost("parallelDestinationCollection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) + .withCollectionZkHost("parallelDestinationCollection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("update", UpdateStream.class) .withFunctionName("parallel", ParallelStream.class) @@ -2711,11 +2616,11 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { params.put("action","list"); int workersComplete = 0; - for(CloudJettyRunner jetty : this.cloudJettys) { + for(JettySolrRunner jetty : cluster.getJettySolrRunners()) { int iterations = 0; INNER: while(iterations == 0) { - SolrStream solrStream = new SolrStream(jetty.url, params); + SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/collection1", params); solrStream.open(); Tuple tupleResponse = solrStream.read(); if (tupleResponse.EOF) { @@ -2738,17 +2643,17 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { } } - assert(workersComplete == 2); + assertEquals(cluster.getJettySolrRunners().size(), workersComplete); - destinationCollectionClient.commit(); + cluster.getSolrClient().commit("parallelDestinationCollection1"); //Lets stop the daemons params = new HashMap(); params.put(CommonParams.QT,"/stream"); params.put("action", "stop"); params.put("id", "test"); - for(CloudJettyRunner jetty : this.cloudJettys) { - SolrStream solrStream = new SolrStream(jetty.url, params); + for (JettySolrRunner jetty : cluster.getJettySolrRunners()) { + SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", params); solrStream.open(); Tuple tupleResponse = solrStream.read(); solrStream.close(); @@ -2759,11 +2664,11 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { params.put("action","list"); workersComplete = 0; - for(CloudJettyRunner jetty : this.cloudJettys) { + for (JettySolrRunner jetty : cluster.getJettySolrRunners()) { long stopTime = 0; INNER: while(stopTime == 0) { - SolrStream solrStream = new SolrStream(jetty.url, params); + SolrStream solrStream = new SolrStream(jetty.getBaseUrl() + "/collection1", params); solrStream.open(); Tuple tupleResponse = solrStream.read(); if (tupleResponse.EOF) { @@ -2785,7 +2690,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { } } - assertTrue(workersComplete == 2); + assertEquals(cluster.getJettySolrRunners().size(), workersComplete); //Ensure that destinationCollection actually has the new docs. expression = StreamExpressionParser.parse("search(parallelDestinationCollection1, q=*:*, fl=\"id,a_s,a_i,a_f,s_multi,i_multi\", sort=\"a_i asc\")"); stream = new CloudSolrStream(expression, factory); @@ -2832,34 +2737,30 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assertList(tuple.getStrings("s_multi"), "aaaa3", "bbbb3"); assertList(tuple.getLongs("i_multi"), Long.parseLong("4444"), Long.parseLong("7777")); - destinationCollectionClient.deleteByQuery("*:*"); - destinationCollectionClient.commit(); - destinationCollectionClient.close(); - del("*:*"); - commit(); } + @Test + public void testIntersectStream() throws Exception { - - private void testIntersectStream() throws Exception{ - indexr(id, "0", "a_s", "setA", "a_i", "0"); - indexr(id, "2", "a_s", "setA", "a_i", "1"); - indexr(id, "3", "a_s", "setA", "a_i", "2"); - indexr(id, "4", "a_s", "setA", "a_i", "3"); - - indexr(id, "5", "a_s", "setB", "a_i", "2"); - indexr(id, "6", "a_s", "setB", "a_i", "3"); - - indexr(id, "7", "a_s", "setAB", "a_i", "0"); - indexr(id, "8", "a_s", "setAB", "a_i", "6"); - commit(); + new UpdateRequest() + .add(id, "0", "a_s", "setA", "a_i", "0") + .add(id, "2", "a_s", "setA", "a_i", "1") + .add(id, "3", "a_s", "setA", "a_i", "2") + .add(id, "4", "a_s", "setA", "a_i", "3") + + .add(id, "5", "a_s", "setB", "a_i", "2") + .add(id, "6", "a_s", "setB", "a_i", "3") + + .add(id, "7", "a_s", "setAB", "a_i", "0") + .add(id, "8", "a_s", "setAB", "a_i", "6") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("intersect", IntersectStream.class); @@ -2873,32 +2774,33 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 5); assertOrder(tuples, 0,7,3,4,8); - - del("*:*"); - commit(); + } - - private void testParallelIntersectStream() throws Exception { - indexr(id, "0", "a_s", "setA", "a_i", "0"); - indexr(id, "2", "a_s", "setA", "a_i", "1"); - indexr(id, "3", "a_s", "setA", "a_i", "2"); - indexr(id, "4", "a_s", "setA", "a_i", "3"); - - indexr(id, "5", "a_s", "setB", "a_i", "2"); - indexr(id, "6", "a_s", "setB", "a_i", "3"); - - indexr(id, "7", "a_s", "setAB", "a_i", "0"); - indexr(id, "8", "a_s", "setAB", "a_i", "6"); - commit(); + + @Test + public void testParallelIntersectStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_s", "setA", "a_i", "0") + .add(id, "2", "a_s", "setA", "a_i", "1") + .add(id, "3", "a_s", "setA", "a_i", "2") + .add(id, "4", "a_s", "setA", "a_i", "3") + + .add(id, "5", "a_s", "setB", "a_i", "2") + .add(id, "6", "a_s", "setB", "a_i", "3") + + .add(id, "7", "a_s", "setAB", "a_i", "0") + .add(id, "8", "a_s", "setAB", "a_i", "6") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory streamFactory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("intersect", IntersectStream.class) .withFunctionName("parallel", ParallelStream.class); // basic - String zkHost = zkServer.getZkAddress(); + String zkHost = cluster.getZkServer().getZkAddress(); final TupleStream stream = streamFactory.constructStream("parallel(" + "collection1, " + "intersect(" @@ -2910,31 +2812,32 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 5); assertOrder(tuples, 0,7,3,4,8); - - del("*:*"); - commit(); + } - - private void testComplementStream() throws Exception{ - indexr(id, "0", "a_s", "setA", "a_i", "0"); - indexr(id, "2", "a_s", "setA", "a_i", "1"); - indexr(id, "3", "a_s", "setA", "a_i", "2"); - indexr(id, "4", "a_s", "setA", "a_i", "3"); - - indexr(id, "5", "a_s", "setB", "a_i", "2"); - indexr(id, "6", "a_s", "setB", "a_i", "3"); - indexr(id, "9", "a_s", "setB", "a_i", "5"); - - indexr(id, "7", "a_s", "setAB", "a_i", "0"); - indexr(id, "8", "a_s", "setAB", "a_i", "6"); - commit(); + + @Test + public void testComplementStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_s", "setA", "a_i", "0") + .add(id, "2", "a_s", "setA", "a_i", "1") + .add(id, "3", "a_s", "setA", "a_i", "2") + .add(id, "4", "a_s", "setA", "a_i", "3") + + .add(id, "5", "a_s", "setB", "a_i", "2") + .add(id, "6", "a_s", "setB", "a_i", "3") + .add(id, "9", "a_s", "setB", "a_i", "5") + + .add(id, "7", "a_s", "setAB", "a_i", "0") + .add(id, "8", "a_s", "setAB", "a_i", "6") + .commit(cluster.getSolrClient(), COLLECTION); StreamExpression expression; TupleStream stream; List tuples; StreamFactory factory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("complement", ComplementStream.class); @@ -2948,32 +2851,33 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 1); assertOrder(tuples, 2); - - del("*:*"); - commit(); + } - - private void testParallelComplementStream() throws Exception { - indexr(id, "0", "a_s", "setA", "a_i", "0"); - indexr(id, "2", "a_s", "setA", "a_i", "1"); - indexr(id, "3", "a_s", "setA", "a_i", "2"); - indexr(id, "4", "a_s", "setA", "a_i", "3"); - - indexr(id, "5", "a_s", "setB", "a_i", "2"); - indexr(id, "6", "a_s", "setB", "a_i", "3"); - indexr(id, "9", "a_s", "setB", "a_i", "5"); - - indexr(id, "7", "a_s", "setAB", "a_i", "0"); - indexr(id, "8", "a_s", "setAB", "a_i", "6"); - commit(); + + @Test + public void testParallelComplementStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_s", "setA", "a_i", "0") + .add(id, "2", "a_s", "setA", "a_i", "1") + .add(id, "3", "a_s", "setA", "a_i", "2") + .add(id, "4", "a_s", "setA", "a_i", "3") + + .add(id, "5", "a_s", "setB", "a_i", "2") + .add(id, "6", "a_s", "setB", "a_i", "3") + .add(id, "9", "a_s", "setB", "a_i", "5") + + .add(id, "7", "a_s", "setAB", "a_i", "0") + .add(id, "8", "a_s", "setAB", "a_i", "6") + .commit(cluster.getSolrClient(), COLLECTION); StreamFactory streamFactory = new StreamFactory() - .withCollectionZkHost("collection1", zkServer.getZkAddress()) + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) .withFunctionName("search", CloudSolrStream.class) .withFunctionName("complement", ComplementStream.class) .withFunctionName("parallel", ParallelStream.class); - final String zkHost = zkServer.getZkAddress(); + final String zkHost = cluster.getZkServer().getZkAddress(); final TupleStream stream = streamFactory.constructStream("parallel(" + "collection1, " + "complement(" @@ -2985,9 +2889,7 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 1); assertOrder(tuples, 2); - - del("*:*"); - commit(); + } protected List getTuples(TupleStream tupleStream) throws IOException { @@ -3120,10 +3022,4 @@ public class StreamExpressionTest extends AbstractFullDistribZkTestBase { return true; } - - @Override - protected void indexr(Object... fields) throws Exception { - SolrInputDocument doc = getDoc(fields); - indexDoc(doc); - } } diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java index 61253e1d9ee..9db02ebadf4 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamingTest.java @@ -16,22 +16,19 @@ */ package org.apache.solr.client.solrj.io.stream; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import org.apache.lucene.util.LuceneTestCase; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.io.SolrClientCache; +import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; -import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator; import org.apache.solr.client.solrj.io.comp.FieldComparator; -import org.apache.solr.client.solrj.io.comp.StreamComparator; +import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator; import org.apache.solr.client.solrj.io.eq.FieldEqualitor; import org.apache.solr.client.solrj.io.ops.GroupOperation; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; @@ -42,13 +39,13 @@ import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric; import org.apache.solr.client.solrj.io.stream.metrics.Metric; import org.apache.solr.client.solrj.io.stream.metrics.MinMetric; import org.apache.solr.client.solrj.io.stream.metrics.SumMetric; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.common.SolrInputDocument; -import org.junit.After; -import org.junit.AfterClass; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.cloud.AbstractDistribZkTestBase; +import org.apache.solr.cloud.SolrCloudTestCase; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; /** @@ -57,106 +54,70 @@ import org.junit.Test; * **/ -@Slow @LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"}) -public class StreamingTest extends AbstractFullDistribZkTestBase { +public class StreamingTest extends SolrCloudTestCase { - private static final String SOLR_HOME = getFile("solrj" + File.separator + "solr").getAbsolutePath(); - private StreamFactory streamFactory; + public static final int TIMEOUT = 30; - static { - schemaString = "schema-streaming.xml"; - } + public static final String COLLECTION = "streams"; + + private static final StreamFactory streamFactory = new StreamFactory() + .withFunctionName("search", CloudSolrStream.class) + .withFunctionName("merge", MergeStream.class) + .withFunctionName("unique", UniqueStream.class) + .withFunctionName("top", RankStream.class) + .withFunctionName("reduce", ReducerStream.class) + .withFunctionName("group", GroupOperation.class) + .withFunctionName("rollup", RollupStream.class) + .withFunctionName("parallel", ParallelStream.class); + + private static String zkHost; @BeforeClass - public static void beforeSuperClass() { - AbstractZkTestCase.SOLRHOME = new File(SOLR_HOME()); + public static void configureCluster() throws Exception { + configureCluster(2) + .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf")) + .configure(); + + CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1).process(cluster.getSolrClient()); + AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), false, true, TIMEOUT); + + zkHost = cluster.getZkServer().getZkAddress(); + streamFactory.withCollectionZkHost(COLLECTION, zkHost); } - @AfterClass - public static void afterSuperClass() { - - } - - protected String getCloudSolrConfig() { - return "solrconfig-streaming.xml"; - } - - - @Override - public String getSolrHome() { - return SOLR_HOME; - } - - public static String SOLR_HOME() { - return SOLR_HOME; - } + private static final String id = "id"; @Before - @Override - public void setUp() throws Exception { - super.setUp(); - // we expect this time of exception as shards go up and down... - //ignoreException(".*"); - //System.setProperty("export.test", "true"); - System.setProperty("numShards", Integer.toString(sliceCount)); + public void clearCollection() throws Exception { + new UpdateRequest() + .deleteByQuery("*:*") + .commit(cluster.getSolrClient(), COLLECTION); } - @Override - @After - public void tearDown() throws Exception { - super.tearDown(); - resetExceptionIgnores(); - } - - public StreamingTest() { - super(); - sliceCount = 2; - - streamFactory = new StreamFactory() - .withFunctionName("search", CloudSolrStream.class) - .withFunctionName("merge", MergeStream.class) - .withFunctionName("unique", UniqueStream.class) - .withFunctionName("top", RankStream.class) - .withFunctionName("reduce", ReducerStream.class) - .withFunctionName("group", GroupOperation.class) - .withFunctionName("rollup", RollupStream.class) - .withFunctionName("parallel", ParallelStream.class); - } - - private void testUniqueStream() throws Exception { + @Test + public void testUniqueStream() throws Exception { //Test CloudSolrStream and UniqueStream - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - - commit(); - - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f")); List tuples = getTuples(ustream); - assert(tuples.size() == 4); + assertEquals(4, tuples.size()); assertOrder(tuples, 0,1,3,4); - del("*:*"); - commit(); - } - - private void testSpacesInParams() throws Exception { - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + @Test + public void testSpacesInParams() throws Exception { Map params = mapParams("q", "*:*", "fl", "id , a_s , a_i , a_f", "sort", "a_f asc , a_i asc"); @@ -164,66 +125,55 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //The constructor will throw an exception if the sort fields do not the //a value in the field list. - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); - - del("*:*"); - commit(); - + CloudSolrStream stream = new CloudSolrStream("", "collection1", params); } - private void testNonePartitionKeys() throws Exception { + @Test + public void testNonePartitionKeys() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "*:*", "fl", "id,a_s,a_i,a_f", "sort", "a_s asc,a_f asc", "partitionKeys", "none"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", stream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING)); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); assert(tuples.size() == 20); // Each tuple will be double counted. - del("*:*"); - commit(); - } - private void testParallelUniqueStream() throws Exception { + @Test + public void testParallelUniqueStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello1", "a_i", "10", "a_f", "1"); - indexr(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5"); - indexr(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5"); - indexr(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello1", "a_i", "10", "a_f", "1") + .add(id, "6", "a_s", "hello1", "a_i", "11", "a_f", "5") + .add(id, "7", "a_s", "hello1", "a_i", "12", "a_f", "5") + .add(id, "8", "a_s", "hello1", "a_i", "13", "a_f", "4") + .commit(cluster.getSolrClient(), COLLECTION); Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc", "partitionKeys", "a_f"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); UniqueStream ustream = new UniqueStream(stream, new FieldEqualitor("a_f")); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", ustream, 2, new FieldComparator("a_f",ComparatorOrder.ASCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, ustream, 2, new FieldComparator("a_f",ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); assert(tuples.size() == 5); @@ -234,129 +184,104 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { Map eofTuples = pstream.getEofTuples(); assert(eofTuples.size() == 2); //There should be an EOF tuple for each worker. - del("*:*"); - commit(); - } + @Test + public void testRankStream() throws Exception { + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); - private void testRankStream() throws Exception { - - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); RankStream rstream = new RankStream(stream, 3, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); List tuples = getTuples(rstream); - assert(tuples.size() == 3); assertOrder(tuples, 4,3,2); - del("*:*"); - commit(); } - private void testParallelRankStream() throws Exception { + @Test + public void testParallelRankStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "5", "a_s", "hello1", "a_i", "5", "a_f", "1"); - indexr(id, "6", "a_s", "hello1", "a_i", "6", "a_f", "1"); - indexr(id, "7", "a_s", "hello1", "a_i", "7", "a_f", "1"); - indexr(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1"); - indexr(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1"); - indexr(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "5", "a_s", "hello1", "a_i", "5", "a_f", "1") + .add(id, "6", "a_s", "hello1", "a_i", "6", "a_f", "1") + .add(id, "7", "a_s", "hello1", "a_i", "7", "a_f", "1") + .add(id, "8", "a_s", "hello1", "a_i", "8", "a_f", "1") + .add(id, "9", "a_s", "hello1", "a_i", "9", "a_f", "1") + .add(id, "10", "a_s", "hello1", "a_i", "10", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); Map params = mapParams("q", "*:*", "fl", "id,a_s,a_i", "sort", "a_i asc", "partitionKeys", "a_i"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); RankStream rstream = new RankStream(stream, 11, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, rstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); assert(tuples.size() == 10); assertOrder(tuples, 10,9,8,7,6,5,4,3,2,0); - del("*:*"); - commit(); } - private void testTrace() throws Exception { + @Test + public void testTrace() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Test with spaces in the parameter lists. Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); stream.setTrace(true); List tuples = getTuples(stream); - assert(tuples.get(0).get("_COLLECTION_").equals("collection1")); - assert(tuples.get(1).get("_COLLECTION_").equals("collection1")); - assert(tuples.get(2).get("_COLLECTION_").equals("collection1")); - assert(tuples.get(3).get("_COLLECTION_").equals("collection1")); - - del("*:*"); - commit(); + assert(tuples.get(0).get("_COLLECTION_").equals(COLLECTION)); + assert(tuples.get(1).get("_COLLECTION_").equals(COLLECTION)); + assert(tuples.get(2).get("_COLLECTION_").equals(COLLECTION)); + assert(tuples.get(3).get("_COLLECTION_").equals(COLLECTION)); } + @Test + public void testReducerStream() throws Exception { - - - private void testReducerStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Test with spaces in the parameter lists. Map paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); ReducerStream rstream = new ReducerStream(stream, new FieldEqualitor("a_s"), new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5)); @@ -379,7 +304,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test with spaces in the parameter lists using a comparator paramsA = mapParams("q","*:*","fl","id,a_s, a_i, a_f","sort", "a_s asc , a_f asc"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); rstream = new ReducerStream(stream, new FieldComparator("a_s", ComparatorOrder.ASCENDING), new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5)); @@ -400,32 +325,28 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { maps2 = t2.getMaps("group"); assertMaps(maps2, 6, 4); - del("*:*"); - commit(); } - private void testZeroReducerStream() throws Exception { + @Test + public void testZeroReducerStream() throws Exception { //Gracefully handle zero results - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Test with spaces in the parameter lists. Map paramsA = mapParams("q", "blah", "fl", "id,a_s, a_i, a_f", "sort", "a_s asc , a_f asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); ReducerStream rstream = new ReducerStream(stream, new FieldEqualitor("a_s"), new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 5)); @@ -434,37 +355,32 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 0); - del("*:*"); - commit(); } + @Test + public void testParallelReducerStream() throws Exception { - private void testParallelReducerStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); ReducerStream rstream = new ReducerStream(stream, new FieldEqualitor("a_s"), new GroupOperation(new FieldComparator("a_f", ComparatorOrder.DESCENDING), 5)); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, rstream, 2, new FieldComparator("a_s",ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); @@ -486,13 +402,13 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test Descending with Ascending subsort paramsA = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_s desc,a_f asc", "partitionKeys", "a_s"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); rstream = new ReducerStream(stream, new FieldEqualitor("a_s"), new GroupOperation(new FieldComparator("a_f", ComparatorOrder.ASCENDING), 3)); - pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s",ComparatorOrder.DESCENDING)); + pstream = new ParallelStream(zkHost, COLLECTION, rstream, 2, new FieldComparator("a_s",ComparatorOrder.DESCENDING)); attachStreamFactory(pstream); tuples = getTuples(pstream); @@ -511,34 +427,28 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { maps2 = t2.getMaps("group"); assertMaps(maps2, 0, 2, 1); - - - del("*:*"); - commit(); } + @Test + @Ignore + public void testExceptionStream() throws Exception { - private void testExceptionStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); //Test an error that comes originates from the /select handler Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); ExceptionStream estream = new ExceptionStream(stream); Tuple t = getTuple(estream); assert(t.EOF); @@ -547,7 +457,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test an error that comes originates from the /export handler paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt","/export"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); estream = new ExceptionStream(stream); t = getTuple(estream); assert(t.EOF); @@ -556,26 +466,26 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assert(t.getException().contains("undefined field:")); } - private void testParallelExceptionStream() throws Exception { + @Test + @Ignore + public void testParallelExceptionStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); - ParallelStream pstream = new ParallelStream(zkHost,"collection1", stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING)); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING)); ExceptionStream estream = new ExceptionStream(pstream); Tuple t = getTuple(estream); assert(t.EOF); @@ -586,8 +496,8 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test an error that originates from the /select handler paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,blah", "sort", "blah asc", "partitionKeys","a_s"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); - pstream = new ParallelStream(zkHost,"collection1", stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING)); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); + pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("blah", ComparatorOrder.ASCENDING)); estream = new ExceptionStream(pstream); t = getTuple(estream); assert(t.EOF); @@ -597,8 +507,8 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test an error that originates from the /export handler paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f,score", "sort", "a_s asc", "qt","/export", "partitionKeys","a_s"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); - pstream = new ParallelStream(zkHost,"collection1", stream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); + pstream = new ParallelStream(zkHost, COLLECTION, stream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); estream = new ExceptionStream(pstream); t = getTuple(estream); assert(t.EOF); @@ -607,23 +517,21 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assert(t.getException().contains("undefined field:")); } + @Test + public void testStatsStream() throws Exception { - private void testStatsStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "*:*"); @@ -637,10 +545,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { new MeanMetric("a_f"), new CountMetric()}; - StatsStream statsStream = new StatsStream(zkHost, - "collection1", - paramsA, - metrics); + StatsStream statsStream = new StatsStream(zkHost, COLLECTION, paramsA, metrics); List tuples = getTuples(statsStream); @@ -670,26 +575,23 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 10); - del("*:*"); - commit(); } - private void testFacetStream() throws Exception { + @Test + public void testFacetStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc"); @@ -708,13 +610,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { FieldComparator[] sorts = {new FieldComparator("sum(a_i)", ComparatorOrder.ASCENDING)}; - FacetStream facetStream = new FacetStream(zkHost, - "collection1", - paramsA, - buckets, - metrics, - sorts, - 100); + FacetStream facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100); List tuples = getTuples(facetStream); @@ -796,13 +692,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { sorts[0] = new FieldComparator("sum(a_i)", ComparatorOrder.DESCENDING); - facetStream = new FacetStream(zkHost, - "collection1", - paramsA, - buckets, - metrics, - sorts, - 100); + facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100); tuples = getTuples(facetStream); @@ -885,13 +775,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { sorts[0] = new FieldComparator("a_s", ComparatorOrder.DESCENDING); - facetStream = new FacetStream(zkHost, - "collection1", - paramsA, - buckets, - metrics, - sorts, - 100); + facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100); tuples = getTuples(facetStream); @@ -922,7 +806,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - tuple = tuples.get(1); bucket = tuple.getString("a_s"); sumi = tuple.getDouble("sum(a_i)"); @@ -946,7 +829,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 6.5D); assertTrue(count.doubleValue() == 4); - tuple = tuples.get(2); bucket = tuple.getString("a_s"); sumi = tuple.getDouble("sum(a_i)"); @@ -974,19 +856,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { sorts[0] = new FieldComparator("a_s", ComparatorOrder.ASCENDING); - facetStream = new FacetStream(zkHost, - "collection1", - paramsA, - buckets, - metrics, - sorts, - 100); + facetStream = new FacetStream(zkHost, COLLECTION, paramsA, buckets, metrics, sorts, 100); tuples = getTuples(facetStream); assert(tuples.size() == 3); - tuple = tuples.get(0); bucket = tuple.getString("a_s"); sumi = tuple.getDouble("sum(a_i)"); @@ -1010,7 +885,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 4.5D); assertTrue(count.doubleValue() == 4); - tuple = tuples.get(1); bucket = tuple.getString("a_s"); sumi = tuple.getDouble("sum(a_i)"); @@ -1034,7 +908,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 6.5D); assertTrue(count.doubleValue() == 4); - tuple = tuples.get(2); bucket = tuple.getString("a_s"); sumi = tuple.getDouble("sum(a_i)"); @@ -1058,27 +931,23 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } + @Test + public void testSubFacetStream() throws Exception { - private void testSubFacetStream() throws Exception { - - indexr(id, "0", "level1_s", "hello0", "level2_s", "a", "a_i", "0", "a_f", "1"); - indexr(id, "2", "level1_s", "hello0", "level2_s", "a", "a_i", "2", "a_f", "2"); - indexr(id, "3", "level1_s", "hello3", "level2_s", "a", "a_i", "3", "a_f", "3"); - indexr(id, "4", "level1_s", "hello4", "level2_s", "a", "a_i", "4", "a_f", "4"); - indexr(id, "1", "level1_s", "hello0", "level2_s", "b", "a_i", "1", "a_f", "5"); - indexr(id, "5", "level1_s", "hello3", "level2_s", "b", "a_i", "10", "a_f", "6"); - indexr(id, "6", "level1_s", "hello4", "level2_s", "b", "a_i", "11", "a_f", "7"); - indexr(id, "7", "level1_s", "hello3", "level2_s", "b", "a_i", "12", "a_f", "8"); - indexr(id, "8", "level1_s", "hello3", "level2_s", "b", "a_i", "13", "a_f", "9"); - indexr(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); + new UpdateRequest() + .add(id, "0", "level1_s", "hello0", "level2_s", "a", "a_i", "0", "a_f", "1") + .add(id, "2", "level1_s", "hello0", "level2_s", "a", "a_i", "2", "a_f", "2") + .add(id, "3", "level1_s", "hello3", "level2_s", "a", "a_i", "3", "a_f", "3") + .add(id, "4", "level1_s", "hello4", "level2_s", "a", "a_i", "4", "a_f", "4") + .add(id, "1", "level1_s", "hello0", "level2_s", "b", "a_i", "1", "a_f", "5") + .add(id, "5", "level1_s", "hello3", "level2_s", "b", "a_i", "10", "a_f", "6") + .add(id, "6", "level1_s", "hello4", "level2_s", "b", "a_i", "11", "a_f", "7") + .add(id, "7", "level1_s", "hello3", "level2_s", "b", "a_i", "12", "a_f", "8") + .add(id, "8", "level1_s", "hello3", "level2_s", "b", "a_i", "13", "a_f", "9") + .add(id, "9", "level1_s", "hello0", "level2_s", "b", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q","*:*","fl","a_i,a_f"); @@ -1089,10 +958,9 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { FieldComparator[] sorts = {new FieldComparator("sum(a_i)", ComparatorOrder.DESCENDING), new FieldComparator("sum(a_i)", ComparatorOrder.DESCENDING)}; - FacetStream facetStream = new FacetStream( zkHost, - "collection1", + COLLECTION, paramsA, buckets, metrics, @@ -1172,7 +1040,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { sorts[1] = new FieldComparator("level2_s", ComparatorOrder.DESCENDING ); facetStream = new FacetStream( zkHost, - "collection1", + COLLECTION, paramsA, buckets, metrics, @@ -1248,29 +1116,26 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(sumi.longValue() == 2); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } - private void testRollupStream() throws Exception { + @Test + public void testRollupStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q","*:*","fl","a_s,a_i,a_f","sort", "a_s asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); Bucket[] buckets = {new Bucket("a_s")}; @@ -1365,11 +1230,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test will null value in the grouping field - indexr(id, "12", "a_s", null, "a_i", "14", "a_f", "10"); - commit(); + new UpdateRequest() + .add(id, "12", "a_s", null, "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); paramsA = mapParams("q","*:*","fl","a_s,a_i,a_f","sort", "a_s asc", "qt", "/export"); - stream = new CloudSolrStream(zkHost, "collection1", paramsA); + stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); Bucket[] buckets1 = {new Bucket("a_s")}; @@ -1410,15 +1276,10 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 10.0D); assertTrue(count.doubleValue() == 1); - - del("*:*"); - commit(); } - - private void testDaemonTopicStream() throws Exception { - - String zkHost = zkServer.getZkAddress(); + @Test + public void testDaemonTopicStream() throws Exception { StreamContext context = new StreamContext(); SolrClientCache cache = new SolrClientCache(); @@ -1429,7 +1290,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { params.put("rows", "500"); params.put("fl", "id"); - TopicStream topicStream = new TopicStream(zkHost, "collection1", "collection1", "50000000", 1000000, params); + TopicStream topicStream = new TopicStream(zkHost, COLLECTION, COLLECTION, "50000000", 1000000, params); DaemonStream daemonStream = new DaemonStream(topicStream, "daemon1", 1000, 500); daemonStream.setStreamContext(context); @@ -1437,7 +1298,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { daemonStream.open(); // Wait for the checkpoint - CloudJettyRunner jetty = this.cloudJettys.get(0); + JettySolrRunner jetty = cluster.getJettySolrRunners().get(0); Map params1 = new HashMap(); params1.put("qt","/get"); @@ -1445,7 +1306,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { params1.put("fl","id"); int count = 0; while(count == 0) { - SolrStream solrStream = new SolrStream(jetty.url, params1); + SolrStream solrStream = new SolrStream(jetty.getBaseUrl().toString() + "/" + COLLECTION, params1); List tuples = getTuples(solrStream); count = tuples.size(); if(count > 0) { @@ -1456,24 +1317,22 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { } } - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello0", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello0", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - - commit(); - + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello0", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello0", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .commit(cluster.getSolrClient(), COLLECTION); for(int i=0; i<5; i++) { daemonStream.read(); } - - indexr(id, "5", "a_s", "hello0", "a_i", "4", "a_f", "4"); - indexr(id, "6", "a_s", "hello0", "a_i", "4", "a_f", "4"); - - commit(); + new UpdateRequest() + .add(id, "5", "a_s", "hello0", "a_i", "4", "a_f", "4") + .add(id, "6", "a_s", "hello0", "a_i", "4", "a_f", "4") + .commit(cluster.getSolrClient(), COLLECTION); for(int i=0; i<2; i++) { daemonStream.read(); @@ -1486,30 +1345,27 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(tuple.EOF); daemonStream.close(); cache.close(); - del("*:*"); - commit(); + } - private void testParallelRollupStream() throws Exception { + @Test + public void testParallelRollupStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "*:*", "fl", "a_s,a_i,a_f", "sort", "a_s asc", "partitionKeys", "a_s"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); Bucket[] buckets = {new Bucket("a_s")}; @@ -1524,7 +1380,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { new CountMetric()}; RollupStream rollupStream = new RollupStream(stream, buckets, metrics); - ParallelStream parallelStream = new ParallelStream(zkHost, "collection1", rollupStream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); + ParallelStream parallelStream = new ParallelStream(zkHost, COLLECTION, rollupStream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); attachStreamFactory(parallelStream); List tuples = getTuples(parallelStream); @@ -1601,55 +1457,48 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertTrue(avgf.doubleValue() == 5.5D); assertTrue(count.doubleValue() == 2); - del("*:*"); - commit(); } - private void testZeroParallelReducerStream() throws Exception { + @Test + public void testZeroParallelReducerStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1"); - indexr(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5"); - indexr(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6"); - indexr(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7"); - indexr(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8"); - indexr(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9"); - indexr(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1") + .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5") + .add(id, "5", "a_s", "hello3", "a_i", "10", "a_f", "6") + .add(id, "6", "a_s", "hello4", "a_i", "11", "a_f", "7") + .add(id, "7", "a_s", "hello3", "a_i", "12", "a_f", "8") + .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9") + .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10") + .commit(cluster.getSolrClient(), COLLECTION); Map paramsA = mapParams("q", "blah", "fl", "id,a_s,a_i,a_f","sort", "a_s asc,a_f asc", "partitionKeys", "a_s"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, paramsA); ReducerStream rstream = new ReducerStream(stream, new FieldEqualitor("a_s"), new GroupOperation(new FieldComparator("a_s", ComparatorOrder.ASCENDING), 2)); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", rstream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, rstream, 2, new FieldComparator("a_s", ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); assert(tuples.size() == 0); - del("*:*"); - commit(); + } + @Test + public void testTuple() throws Exception { - private void testTuple() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "5.1", "s_multi", "a", "s_multi", "b", "i_multi", "1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "5.1", "s_multi", "a", "s_multi", "b", "i_multi", + "1", "i_multi", "2", "f_multi", "1.2", "f_multi", "1.3") + .commit(cluster.getSolrClient(), COLLECTION); Map params = mapParams("q","*:*","fl","id,a_s,a_i,a_f,s_multi,i_multi,f_multi","sort", "a_s asc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); List tuples = getTuples(stream); Tuple tuple = tuples.get(0); @@ -1675,29 +1524,25 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assert(doubleList.get(0).doubleValue() == 1.2); assert(doubleList.get(1).doubleValue() == 1.3); - del("*:*"); - commit(); } - private void testMergeStream() throws Exception { + @Test + public void testMergeStream() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); //Test ascending Map paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i asc"); - CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); Map paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i asc"); - CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); List tuples = getTuples(mstream); @@ -1707,10 +1552,10 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test descending paramsA = mapParams("q","id:(4 1)","fl","id,a_s,a_i","sort", "a_i desc"); - streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); paramsB = mapParams("q","id:(0 2 3)","fl","id,a_s,a_i","sort", "a_i desc"); - streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); tuples = getTuples(mstream); @@ -1721,10 +1566,10 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test compound sort paramsA = mapParams("q","id:(2 4 1)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); - streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); paramsB = mapParams("q","id:(0 3)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); - streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.ASCENDING))); tuples = getTuples(mstream); @@ -1733,10 +1578,10 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assertOrder(tuples, 0,2,1,3,4); paramsA = mapParams("q","id:(2 4 1)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc"); - streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); paramsB = mapParams("q","id:(0 3)","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc"); - streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); mstream = new MergeStream(streamA, streamB, new MultipleFieldComparator(new FieldComparator("a_f",ComparatorOrder.ASCENDING),new FieldComparator("a_i",ComparatorOrder.DESCENDING))); tuples = getTuples(mstream); @@ -1744,38 +1589,33 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { assert(tuples.size() == 5); assertOrder(tuples, 2,0,1,3,4); - del("*:*"); - commit(); } + @Test + public void testParallelMergeStream() throws Exception { - private void testParallelMergeStream() throws Exception { - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0"); - indexr(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0"); - indexr(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3"); - indexr(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4"); - indexr(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0") + .add(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0") + .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3") + .add(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4") + .add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); //Test ascending Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i"); - CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i"); - CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); @@ -1784,50 +1624,46 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test descending paramsA = mapParams("q", "id:(4 1 8 9)", "fl", "id,a_s,a_i", "sort", "a_i desc", "partitionKeys", "a_i"); - streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i desc", "partitionKeys", "a_i"); - streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); - pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); + pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.DESCENDING)); attachStreamFactory(pstream); tuples = getTuples(pstream); assert(tuples.size() == 8); assertOrder(tuples, 9,8,6,4,3,2,1,0); - del("*:*"); - commit(); } - private void testParallelEOF() throws Exception { + @Test + public void testParallelEOF() throws Exception { - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - indexr(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0"); - indexr(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0"); - indexr(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3"); - indexr(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4"); - indexr(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .add(id, "5", "a_s", "hello0", "a_i", "10", "a_f", "0") + .add(id, "6", "a_s", "hello2", "a_i", "8", "a_f", "0") + .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "3") + .add(id, "8", "a_s", "hello4", "a_i", "11", "a_f", "4") + .add(id, "9", "a_s", "hello1", "a_i", "100", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); //Test ascending Map paramsA = mapParams("q","id:(4 1 8 7 9)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i"); - CloudSolrStream streamA = new CloudSolrStream(zkHost, "collection1", paramsA); + CloudSolrStream streamA = new CloudSolrStream(zkHost, COLLECTION, paramsA); Map paramsB = mapParams("q","id:(0 2 3 6)","fl","id,a_s,a_i","sort", "a_i asc", "partitionKeys", "a_i"); - CloudSolrStream streamB = new CloudSolrStream(zkHost, "collection1", paramsB); + CloudSolrStream streamB = new CloudSolrStream(zkHost, COLLECTION, paramsB); MergeStream mstream = new MergeStream(streamA, streamB, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); - ParallelStream pstream = new ParallelStream(zkHost, "collection1", mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); + ParallelStream pstream = new ParallelStream(zkHost, COLLECTION, mstream, 2, new FieldComparator("a_i",ComparatorOrder.ASCENDING)); attachStreamFactory(pstream); List tuples = getTuples(pstream); @@ -1836,41 +1672,25 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { Map eofTuples = pstream.getEofTuples(); assert(eofTuples.size() == 2); // There should be an EOF Tuple for each worker. - del("*:*"); - commit(); } - - @Test public void streamTests() throws Exception { - assertNotNull(cloudClient); - handle.clear(); - handle.put("timestamp", SKIPVAL); + new UpdateRequest() + .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0") + .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0") + .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3") + .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4") + .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1") + .commit(cluster.getSolrClient(), COLLECTION); - waitForRecoveriesToFinish(false); - - del("*:*"); - - commit(); - - indexr(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0"); - indexr(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0"); - indexr(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3"); - indexr(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4"); - indexr(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1"); - - commit(); - - String zkHost = zkServer.getZkAddress(); - streamFactory.withCollectionZkHost("collection1", zkHost); Map params = null; //Basic CloudSolrStream Test with Descending Sort params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i desc"); - CloudSolrStream stream = new CloudSolrStream(zkHost, "collection1", params); + CloudSolrStream stream = new CloudSolrStream(zkHost, COLLECTION, params); List tuples = getTuples(stream); assert(tuples.size() == 5); @@ -1878,7 +1698,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //With Ascending Sort params = mapParams("q","*:*","fl","id,a_s,a_i","sort", "a_i asc"); - stream = new CloudSolrStream(zkHost, "collection1", params); + stream = new CloudSolrStream(zkHost, COLLECTION, params); tuples = getTuples(stream); assert(tuples.size() == 5); @@ -1887,7 +1707,7 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { //Test compound sort params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i desc"); - stream = new CloudSolrStream(zkHost, "collection1", params); + stream = new CloudSolrStream(zkHost, COLLECTION, params); tuples = getTuples(stream); assert(tuples.size() == 5); @@ -1895,38 +1715,12 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { params = mapParams("q","*:*","fl","id,a_s,a_i,a_f","sort", "a_f asc,a_i asc"); - stream = new CloudSolrStream(zkHost, "collection1", params); + stream = new CloudSolrStream(zkHost, COLLECTION, params); tuples = getTuples(stream); assert (tuples.size() == 5); assertOrder(tuples, 0, 2, 1, 3, 4); - del("*:*"); - commit(); - - testTuple(); - testSpacesInParams(); - testNonePartitionKeys(); - testTrace(); - testUniqueStream(); - testRankStream(); - testMergeStream(); - testReducerStream(); - testRollupStream(); - testZeroReducerStream(); - testFacetStream(); - testSubFacetStream(); - testStatsStream(); - //testExceptionStream(); - testDaemonTopicStream(); - testParallelEOF(); - testParallelUniqueStream(); - testParallelRankStream(); - testParallelMergeStream(); - testParallelRollupStream(); - testParallelReducerStream(); - //testParallelExceptionStream(); - testZeroParallelReducerStream(); } protected Map mapParams(String... vals) { @@ -2019,12 +1813,6 @@ public class StreamingTest extends AbstractFullDistribZkTestBase { return true; } - - @Override - protected void indexr(Object... fields) throws Exception { - SolrInputDocument doc = getDoc(fields); - indexDoc(doc); - } private void attachStreamFactory(TupleStream tupleStream) { StreamContext streamContext = new StreamContext(); diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java index e83ffc5bdc2..baddfc2f57b 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java @@ -1353,7 +1353,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes if (slices == null) { throw new RuntimeException("Could not find collection " - + DEFAULT_COLLECTION + " in " + clusterState.getCollections()); + + DEFAULT_COLLECTION + " in " + clusterState.getCollectionsMap().keySet()); } for (CloudJettyRunner cjetty : cloudJettys) { @@ -1916,9 +1916,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes if (collection != null) { cs = clusterState.getCollection(collection).toString(); } else { - Map map = new HashMap<>(); - for (String coll : clusterState.getCollections()) - map.put(coll, clusterState.getCollection(coll)); + Map map = clusterState.getCollectionsMap(); CharArr out = new CharArr(); new JSONWriter(out, 2).write(map); cs = out.toString(); diff --git a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java index 90496186574..6235affc0cc 100644 --- a/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java +++ b/solr/test-framework/src/java/org/apache/solr/util/SSLTestConfig.java @@ -21,6 +21,8 @@ import java.security.KeyManagementException; import java.security.KeyStore; import java.security.KeyStoreException; import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.security.SecureRandomSpi; import java.security.UnrecoverableKeyException; import javax.net.ssl.SSLContext; @@ -40,8 +42,10 @@ import org.apache.solr.client.solrj.embedded.SSLConfig; import org.apache.solr.client.solrj.impl.HttpClientUtil; import org.apache.solr.client.solrj.impl.HttpClientUtil.SchemaRegistryProvider; import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder; + import org.eclipse.jetty.util.resource.Resource; import org.eclipse.jetty.util.security.CertificateUtils; +import org.eclipse.jetty.util.ssl.SslContextFactory; public class SSLTestConfig extends SSLConfig { public static File TEST_KEYSTORE = ExternalPaths.SERVER_HOME == null ? null @@ -82,7 +86,10 @@ public class SSLTestConfig extends SSLConfig { /** * Builds a new SSLContext for HTTP clients to use when communicating with servers which have - * been configured based on the settings of this object. Also explicitly allows the use of self-signed + * been configured based on the settings of this object. + * + * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking + * due to lack of entropy, also explicitly allows the use of self-signed * certificates (since that's what is almost always used during testing). */ public SSLContext buildClientSSLContext() throws KeyManagementException, @@ -91,7 +98,8 @@ public class SSLTestConfig extends SSLConfig { assert isSSLMode(); SSLContextBuilder builder = SSLContexts.custom(); - + builder.setSecureRandom(NullSecureRandom.INSTANCE); + // NOTE: KeyStore & TrustStore are swapped because they are from configured from server perspective... // we are a client - our keystore contains the keys the server trusts, and vice versa builder.loadTrustMaterial(buildKeyStore(getKeyStore(), getKeyStorePassword()), new TrustSelfSignedStrategy()).build(); @@ -104,6 +112,54 @@ public class SSLTestConfig extends SSLConfig { return builder.build(); } + /** + * Builds a new SSLContext for jetty servers which have been configured based on the settings of + * this object. + * + * NOTE: Uses a completely insecure {@link SecureRandom} instance to prevent tests from blocking + * due to lack of entropy, also explicitly allows the use of self-signed + * certificates (since that's what is almost always used during testing). + * almost always used during testing). + */ + public SSLContext buildServerSSLContext() throws KeyManagementException, + UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { + + assert isSSLMode(); + + SSLContextBuilder builder = SSLContexts.custom(); + builder.setSecureRandom(NullSecureRandom.INSTANCE); + + builder.loadKeyMaterial(buildKeyStore(getKeyStore(), getKeyStorePassword()), getKeyStorePassword().toCharArray()); + + if (isClientAuthMode()) { + builder.loadTrustMaterial(buildKeyStore(getTrustStore(), getTrustStorePassword()), new TrustSelfSignedStrategy()).build(); + + } + + return builder.build(); + } + + /** + * Returns an SslContextFactory using {@link #buildServerSSLContext} if SSL should be used, else returns null. + */ + @Override + public SslContextFactory createContextFactory() { + if (!isSSLMode()) { + return null; + } + // else... + + + SslContextFactory factory = new SslContextFactory(false); + try { + factory.setSslContext(buildServerSSLContext()); + } catch (Exception e) { + throw new RuntimeException("ssl context init failure: " + e.getMessage(), e); + } + factory.setNeedClientAuth(isClientAuthMode()); + return factory; + } + /** * Constructs a KeyStore using the specified filename and password */ @@ -202,5 +258,42 @@ public class SSLTestConfig extends SSLConfig { System.clearProperty("javax.net.ssl.trustStore"); System.clearProperty("javax.net.ssl.trustStorePassword"); } - + + /** + * A mocked up instance of SecureRandom that always does the minimal amount of work to generate + * "random" numbers. This is to prevent blocking issues that arise in platform default + * SecureRandom instances due to too many instances / not enough random entropy. + * Tests do not need secure SSL. + */ + private static class NullSecureRandom extends SecureRandom { + public static final SecureRandom INSTANCE = new NullSecureRandom(); + + /** SPI Used to init all instances */ + private static final SecureRandomSpi NULL_SPI = new SecureRandomSpi() { + /** NOOP: returns new uninitialized byte[] */ + public byte[] engineGenerateSeed(int numBytes) { + return new byte[numBytes]; + } + /** NOOP */ + public void engineNextBytes(byte[] bytes) { /* NOOP */ } + /** NOOP */ + public void engineSetSeed(byte[] seed) { /* NOOP */ } + }; + + private NullSecureRandom() { + super(NULL_SPI, null) ; + } + + /** NOOP: returns new uninitialized byte[] */ + public byte[] generateSeed(int numBytes) { + return new byte[numBytes]; + } + /** NOOP */ + synchronized public void nextBytes(byte[] bytes) { /* NOOP */ } + /** NOOP */ + synchronized public void setSeed(byte[] seed) { /* NOOP */ } + /** NOOP */ + synchronized public void setSeed(long seed) { /* NOOP */ } + + } }