From 53318ca8b79a593a05ebff130420a4d111e246e2 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Sat, 22 Jul 2017 14:38:07 +0700 Subject: [PATCH 01/14] [BEAM-2657] Create Solr IO --- .../sdk/io/common/IOTestPipelineOptions.java | 6 + sdks/java/io/pom.xml | 1 + sdks/java/io/solr/pom.xml | 151 +++++ .../org/apache/beam/sdk/io/solr/SolrIO.java | 611 ++++++++++++++++++ .../apache/beam/sdk/io/solr/package-info.java | 20 + .../beam/sdk/io/solr/BeamThreadsFilter.java | 13 + .../beam/sdk/io/solr/SolrIOCoderTest.java | 58 ++ .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 174 +++++ .../apache/beam/sdk/io/solr/SolrIOTest.java | 225 +++++++ .../beam/sdk/io/solr/SolrIOTestUtils.java | 107 +++ .../resources/cloud-minimal/conf/schema.xml | 29 + .../cloud-minimal/conf/solrconfig.xml | 48 ++ 12 files changed, 1443 insertions(+) create mode 100644 sdks/java/io/solr/pom.xml create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java create mode 100644 sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml create mode 100644 sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java index 25ab9298ea2f..d317be9427bf 100644 --- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java +++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java @@ -71,6 +71,12 @@ public interface IOTestPipelineOptions extends TestPipelineOptions { Integer getElasticsearchHttpPort(); void setElasticsearchHttpPort(Integer value); + /* Solr */ + @Description("Address of Zookeeper server for Solr") + @Default.String("zookeeper-server") + String getZookeeperSolrServer(); + void setZookeeperSolrServer(String value); + /* Cassandra */ @Description("Host for Cassandra server (host name/ip address)") @Default.String("cassandra-host") diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 4e02aa8286f1..c291e5d1d3cf 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -56,6 +56,7 @@ kinesis mongodb mqtt + solr xml diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml new file mode 100644 index 000000000000..0fd2fa9a3774 --- /dev/null +++ b/sdks/java/io/solr/pom.xml @@ -0,0 +1,151 @@ + + + + beam-sdks-java-io-parent + org.apache.beam + 2.2.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + beam-sdks-java-io-solr + Apache Beam :: SDKs :: Java :: IO :: Solr + IO to read and write on Solr. + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce + + enforce + + + + + 1.8 + + + jdk.tools:jdk.tools + + + + [1.8,) + + + + + + + + org.codehaus.mojo + extra-enforcer-rules + 1.0-beta-6 + + + + + + + + + org.apache.beam + beam-sdks-java-core + + + + com.google.guava + guava + + + + org.apache.solr + solr-solrj + 6.6.0 + + + + com.google.code.findbugs + jsr305 + + + + joda-time + joda-time + + + + org.slf4j + slf4j-api + + + + + com.google.auto.value + auto-value + provided + + + + + + org.hamcrest + hamcrest-core + test + + + + org.hamcrest + hamcrest-all + test + + + + junit + junit + test + + + + org.apache.beam + beam-sdks-java-io-common + test + tests + + + + org.apache.beam + beam-runners-direct-java + test + + + + org.apache.solr + solr-test-framework + 6.6.0 + test + + + + com.carrotsearch.randomizedtesting + randomizedtesting-runner + 2.5.0 + test + + + + org.slf4j + slf4j-log4j12 + 1.7.10 + test + + + + \ No newline at end of file diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java new file mode 100644 index 000000000000..518ca272b109 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -0,0 +1,611 @@ +package org.apache.beam.sdk.io.solr; + + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.net.MalformedURLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +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.impl.CloudSolrClient; +import org.apache.solr.client.solrj.impl.HttpSolrClient; +import org.apache.solr.client.solrj.request.CoreAdminRequest; +import org.apache.solr.client.solrj.response.CoreAdminResponse; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +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.params.CoreAdminParams; +import org.apache.solr.common.params.CursorMarkParams; +import org.apache.solr.common.util.JavaBinCodec; +import org.apache.solr.common.util.NamedList; + +/** + * Transforms for reading and writing data from/to Solr. + * + *

Reading from Solr

+ * + *

{@link SolrIO#read SolrIO.read()} returns a bounded + * {@link PCollection PCollection<SolrDocument>} representing Solr documents. + * + *

To configure the {@link SolrIO#read}, you have to provide a connection configuration + * containing the Zookeeper address of the Solr cluster. The following example + * illustrates options for configuring the source: + * + *

{@code
+ *
+ * pipeline.apply(SolrIO.read().withConnectionConfiguration(
+ *    SolrIO.ConnectionConfiguration.create("127.0.0.1:9983", "my-collection")
+ * )
+ *
+ * }
+ * + *

You can specify a query on the {@code read()} using {@code withQuery()}. + * + *

Writing to Solr

+ * + *

To write documents to Solr, use + * {@link SolrIO#write SolrIO.write()}, which writes Solr documents from a + * {@link PCollection PCollection<SolrInputDocument>} (which can be bounded or unbounded). + * + *

To configure {@link SolrIO#write SolrIO.write()}, similar to the read, you + * have to provide a connection configuration. For instance: + * + *

{@code
+ *
+ *  pipeline
+ *    .apply(...)
+ *    .apply(SolrIO.write().withConnectionConfiguration(
+ *       SolrIO.ConnectionConfiguration.create("127.0.0.1:9983", "my-collection")
+ *    )
+ *
+ * }
+ * + *

Optionally, you can provide {@code withBatchSize()} + * to specify the size of the write batch in number of documents. + */ +@Experimental +public class SolrIO { + + public static Read read() { + return new AutoValue_SolrIO_Read.Builder().setBatchSize(100L).build(); + } + + public static Write write() { + return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000L).build(); + } + + private SolrIO() { + } + + /** A POJO describing a connection configuration to Solr. */ + @AutoValue public abstract static class ConnectionConfiguration implements Serializable { + + //TODO add user name password + abstract String getZkHost(); + + abstract String getCollection(); + + abstract Builder builder(); + + @AutoValue.Builder abstract static class Builder { + + abstract Builder setZkHost(String zkHost); + + abstract Builder setCollection(String collection); + + abstract ConnectionConfiguration build(); + } + + /** + * Creates a new Solr connection configuration. + * + * @param zkHost host of zookeeper + * @param collection the collection toward which the requests will be issued + * @return the connection configuration object + * @throws IOException when it fails to connect to Solr + */ + public static ConnectionConfiguration create(String zkHost, String collection) + throws IOException { + checkArgument(zkHost != null, + "ConnectionConfiguration.create(zkHost, collection) " + + "called with null address"); + checkArgument(collection != null, + "ConnectionConfiguration.create(zkHost, collection) " + + "called with null collectioin"); + ConnectionConfiguration connectionConfiguration = new AutoValue_SolrIO_ConnectionConfiguration + .Builder().setZkHost(zkHost).setCollection(collection).build(); + return connectionConfiguration; + } + + private void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("zkHost", getZkHost())); + builder.add(DisplayData.item("collection", getCollection())); + } + + CloudSolrClient createClient() throws MalformedURLException { + CloudSolrClient solrClient = new CloudSolrClient.Builder().withZkHost(getZkHost()).build(); + solrClient.setDefaultCollection(getCollection()); + return solrClient; + } + + HttpSolrClient createClient(String shardUrl) { + HttpSolrClient solrClient = new HttpSolrClient.Builder(shardUrl).build(); + return solrClient; + } + } + + /** A {@link PTransform} reading data from Solr. */ + @AutoValue public abstract static class Read + extends PTransform> { + + private static final long MAX_BATCH_SIZE = 10000L; + + @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); + + @Nullable abstract String getQuery(); + + abstract long getBatchSize(); + + abstract Builder builder(); + + @AutoValue.Builder abstract static class Builder { + + abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); + + abstract Builder setQuery(String query); + + abstract Builder setBatchSize(long batchSize); + + abstract Read build(); + } + + /** + * Provide the Solr connection configuration object. + * + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Read} with connection configuration set + */ + public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "SolrIO.read()" + + ".withConnectionConfiguration(configuration) called with null configuration"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + + /** + * Provide a query used while reading from Solr. + * + * @param query the query. See + * Solr Query + * + * @return the {@link Read} object with query set + */ + public Read withQuery(String query) { + checkArgument(!Strings.isNullOrEmpty(query), + "SolrIO.read().withQuery(query) called" + " with null or empty query"); + return builder().setQuery(query).build(); + } + + /** + * Provide a size for the cursor read. See + * cursor API Default is 100. Maximum is 10 000. If documents are small, increasing batch + * size might improve read performance. If documents are big, you might need to decrease + * batchSize + * + * @param batchSize number of documents read in each scroll read + * @return the {@link Read} with batch size set + */ + public Read withBatchSize(long batchSize) { + checkArgument(batchSize > 0, "SolrIO.read().withBatchSize(batchSize) " + + "called with a negative or equal to 0 value: %s", batchSize); + checkArgument(batchSize <= MAX_BATCH_SIZE, + "SolrIO.read().withBatchSize(batchSize) " + + "called with a too large value (over %s): %s", + MAX_BATCH_SIZE, batchSize); + return builder().setBatchSize(batchSize).build(); + } + + @Override public PCollection expand(PBegin input) { + return input.apply(org.apache.beam.sdk.io.Read + .from(new BoundedSolrSource(this, null))); + } + + @Override public void validate(PipelineOptions options) { + checkState(getConnectionConfiguration() != null, + "SolrIO.read() requires a connection configuration" + + " to be set via withConnectionConfiguration(configuration)"); + } + + @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotNull(DisplayData.item("query", getQuery())); + getConnectionConfiguration().populateDisplayData(builder); + } + } + + /** A {@link BoundedSource} reading from Solr. */ + @VisibleForTesting static class BoundedSolrSource extends BoundedSource { + + private final SolrIO.Read spec; + // replica is the info of the shard where the source will read the documents + @Nullable private final Replica replica; + + BoundedSolrSource(Read spec, @Nullable Replica replica) { + this.spec = spec; + this.replica = replica; + } + + @Override public List> split(long desiredBundleSizeBytes, + PipelineOptions options) throws Exception { + List sources = new ArrayList<>(); + int numShard; + try (CloudSolrClient client = spec.getConnectionConfiguration().createClient()) { + // connect to zk cluster + client.connect(); + ClusterState clusterState = client.getZkStateReader().getClusterState(); + DocCollection docCollection = clusterState + .getCollection(spec.getConnectionConfiguration().getCollection()); + numShard = docCollection.getSlices().size(); + for (Slice slice : docCollection.getSlices()) { + sources.add(new BoundedSolrSource(spec, slice.getLeader())); + } + } + checkArgument(sources.size() == numShard, "Not enough leaders were found"); + return sources; + } + + @Override public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { + if (replica != null) { + return getEstimatedSizeOfShard(replica); + } else { + return getEstimatedSizeOfCollection(); + } + } + + private long getEstimatedSizeOfShard(Replica replica) throws IOException { + try (HttpSolrClient solrClient = spec.getConnectionConfiguration() + .createClient(replica.getBaseUrl())) { + CoreAdminRequest req = new CoreAdminRequest(); + req.setAction(CoreAdminParams.CoreAdminAction.STATUS); + req.setIndexInfoNeeded(true); + CoreAdminResponse response; + try { + response = req.process(solrClient); + } catch (SolrServerException e) { + throw new IOException("Can not get core status from " + replica, e); + } + NamedList coreStatus = response.getCoreStatus(replica.getCoreName()); + NamedList indexStats = (NamedList) coreStatus.get("index"); + return (long) indexStats.get("sizeInBytes"); + } + } + + private long getEstimatedSizeOfCollection() throws IOException { + long sizeInBytes = 0; + ConnectionConfiguration config = spec.getConnectionConfiguration(); + try (CloudSolrClient solrClient = config.createClient()) { + solrClient.connect(); + ClusterState clusterState = solrClient.getZkStateReader().getClusterState(); + DocCollection docCollection = clusterState.getCollection(config.getCollection()); + for (Slice slice : docCollection.getSlices()) { + Replica replica = slice.getLeader(); + sizeInBytes += getEstimatedSizeOfShard(replica); + } + } + return sizeInBytes; + } + + @Override public void populateDisplayData(DisplayData.Builder builder) { + spec.populateDisplayData(builder); + if (replica != null) { + builder.addIfNotNull(DisplayData.item("shardUrl", replica.getCoreUrl())); + } + } + + @Override public BoundedReader createReader(PipelineOptions options) + throws IOException { + return new BoundedSolrReader(this); + } + + @Override public void validate() { + spec.validate(null); + } + + @Override public Coder getDefaultOutputCoder() { + return SolrCoder.of(); + } + } + + /** + * A {@link Coder} that encodes {@link SolrDocument SolrDocument}. + */ + public static class SolrCoder extends Coder { + + private static final SolrCoder INSTANCE = new SolrCoder(); + + public static SolrCoder of() { + return INSTANCE; + } + + @Override public void encode(SolrDocument value, OutputStream outStream) + throws CoderException, IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + JavaBinCodec codec = new JavaBinCodec(); + codec.marshal(value, baos); + + byte[] bytes = baos.toByteArray(); + VarInt.encode(bytes.length, outStream); + outStream.write(bytes); + } + + @Override public SolrDocument decode(InputStream inStream) throws CoderException, IOException { + DataInputStream in = new DataInputStream(inStream); + + int len = VarInt.decodeInt(in); + if (len < 0) { + throw new CoderException("Invalid encoded SolrDocument length: " + len); + } + byte[] bytes = new byte[len]; + in.readFully(bytes); + + JavaBinCodec codec = new JavaBinCodec(); + return (SolrDocument) codec.unmarshal(new ByteArrayInputStream(bytes)); + } + + @Override public List> getCoderArguments() { + return Collections.emptyList(); + } + + @Override public void verifyDeterministic() throws NonDeterministicException { + + } + } + + private static class BoundedSolrReader extends BoundedSource.BoundedReader { + + private final BoundedSolrSource source; + + private SolrClient solrClient; + private SolrDocument current; + private String cursorMark; + private Iterator batchIterator; + private boolean done; + + private BoundedSolrReader(BoundedSolrSource source) { + this.source = source; + this.cursorMark = CursorMarkParams.CURSOR_MARK_START; + } + + @Override public boolean start() throws IOException { + if (source.replica != null) { + solrClient = source.spec.getConnectionConfiguration() + .createClient(source.replica.getCoreUrl()); + } else { + solrClient = source.spec.getConnectionConfiguration().createClient(); + } + + SolrQuery solrParams = getQueryParams(source); + try { + QueryResponse response = solrClient.query(solrParams); + updateCursorMark(response); + return readNextBatchAndReturnFirstDocument(response); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + private SolrQuery getQueryParams(BoundedSolrSource source) { + String query = source.spec.getQuery(); + if (query == null) { + query = "*:*"; + } + SolrQuery solrQuery = new SolrQuery(query); + solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark); + solrQuery.addSort("id", SolrQuery.ORDER.asc); + if (source.replica != null) { + solrQuery.setDistrib(false); + } + return solrQuery; + } + + private void updateCursorMark(QueryResponse response) { + if (cursorMark.equals(response.getNextCursorMark())) { + done = true; + } + cursorMark = response.getNextCursorMark(); + } + + @Override public boolean advance() throws IOException { + if (batchIterator.hasNext()) { + current = batchIterator.next(); + return true; + } else { + SolrQuery solrQuery = getQueryParams(source); + try { + QueryResponse response = solrClient.query(solrQuery); + updateCursorMark(response); + return readNextBatchAndReturnFirstDocument(response); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + } + + private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { + if (response.getResults().isEmpty() || done) { + current = null; + batchIterator = null; + return false; + } + + batchIterator = response.getResults().iterator(); + current = batchIterator.next(); + return true; + } + + @Override public SolrDocument getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current; + } + + @Override public void close() throws IOException { + solrClient.close(); + } + + @Override public BoundedSource getCurrentSource() { + return source; + } + } + + /** A {@link PTransform} writing data to Solr. */ + @AutoValue public abstract static class Write + extends PTransform, PDone> { + + @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); + + abstract long getMaxBatchSize(); + + abstract Builder builder(); + + @AutoValue.Builder abstract static class Builder { + + abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); + + abstract Builder setMaxBatchSize(long maxBatchSize); + + abstract Write build(); + } + + /** + * Provide the Solr connection configuration object. + * + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Write} with connection configuration set + */ + public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "SolrIO.write()" + + ".withConnectionConfiguration(configuration) called with null configuration"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + + /** + * Provide a maximum size in number of documents for the batch. Depending on the + * execution engine, size of bundles may vary, this sets the maximum size. Change this if you + * need to have smaller batch. + * + * @param batchSize maximum batch size in number of documents + * @return the {@link Write} with connection batch size set + */ + public Write withMaxBatchSize(long batchSize) { + checkArgument(batchSize > 0, + "SolrIO.write()" + ".withMaxBatchSize(batchSize) called with incorrect <= 0 value"); + return builder().setMaxBatchSize(batchSize).build(); + } + + @Override public void validate(PipelineOptions options) { + checkState(getConnectionConfiguration() != null, + "SolrIO.write() requires a connection configuration" + + " to be set via withConnectionConfiguration(configuration)"); + } + + @Override public PDone expand(PCollection input) { + input.apply(ParDo.of(new WriteFn(this))); + return PDone.in(input.getPipeline()); + } + + @VisibleForTesting static class WriteFn extends DoFn { + + private final Write spec; + + private transient CloudSolrClient solrClient; + private Collection batch; + + WriteFn(Write spec) { + this.spec = spec; + } + + @Setup public void createClient() throws Exception { + solrClient = spec.getConnectionConfiguration().createClient(); + solrClient.connect(); + } + + @StartBundle public void startBundle(StartBundleContext context) throws Exception { + batch = new ArrayList<>(); + } + + @ProcessElement public void processElement(ProcessContext context) throws Exception { + SolrInputDocument document = context.element(); + batch.add(document); + if (batch.size() >= spec.getMaxBatchSize()) { + flushBatch(); + } + } + + @FinishBundle + public void finishBundle(FinishBundleContext context) throws Exception { + flushBatch(); + } + + private void flushBatch() throws IOException { + if (batch.isEmpty()) { + return; + } + try { + solrClient.add(spec.getConnectionConfiguration().getCollection(), batch); + } catch (SolrServerException e) { + throw new IOException("Error writing to Solr", e); + } finally { + batch.clear(); + } + } + + @Teardown + public void closeClient() throws Exception { + if (solrClient != null) { + solrClient.close(); + } + } + } + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java new file mode 100644 index 000000000000..3d5770e9e14e --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** Transforms for reading and writing from Solr. */ +package org.apache.beam.sdk.io.solr; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java new file mode 100644 index 000000000000..c94c2ff71429 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java @@ -0,0 +1,13 @@ +package org.apache.beam.sdk.io.solr; + +import com.carrotsearch.randomizedtesting.ThreadFilter; + +/** + * Filter out Beam leak threads. + */ +public class BeamThreadsFilter implements ThreadFilter { + + @Override public boolean reject(Thread t) { + return t.getName().startsWith("direct-metrics-counter"); + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java new file mode 100644 index 000000000000..fae67e0fdba5 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java @@ -0,0 +1,58 @@ +package org.apache.beam.sdk.io.solr; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.common.io.CountingInputStream; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.util.UnownedInputStream; +import org.apache.beam.sdk.util.UnownedOutputStream; +import org.apache.solr.common.SolrDocument; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Test case for {@link SolrIO.SolrCoder}. + */ +@RunWith(JUnit4.class) +public class SolrIOCoderTest { + + private static final Coder TEST_CODER = SolrIO.SolrCoder.of(); + + private static final List TEST_VALUES = new ArrayList<>(); + + static { + SolrDocument doc = new SolrDocument(); + doc.put("id", "1"); + doc.put("content", "wheel on the bus"); + doc.put("_version_", 1573597324260671488L); + TEST_VALUES.add(doc); + + doc = new SolrDocument(); + doc.put("id", "2"); + doc.put("content", "goes round and round"); + doc.put("_version_", 1573597324260671489L); + TEST_VALUES.add(doc); + } + + @Test public void testDecodeEncodeEqual() throws Exception { + for (SolrDocument value : TEST_VALUES) { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + TEST_CODER.encode(value, new UnownedOutputStream(os)); + byte[] bytes = os.toByteArray(); + CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(bytes)); + SolrDocument decoded = TEST_CODER.decode(new UnownedInputStream(cis)); + assertThat("consumed bytes equal to encoded bytes", cis.getCount(), + equalTo((long) bytes.length)); + assertThat(decoded.entrySet(), equalTo(value.entrySet())); + assertThat(decoded.getChildDocuments(), equalTo(value.getChildDocuments())); + } + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java new file mode 100644 index 000000000000..7b03994598f5 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solr; + +import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; + +import java.util.List; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.common.IOTestPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SourceTestUtils; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A test of {@link SolrIO} on an independent Solr instance. + * + *

This test requires a running instance of Solr (./bin/solr start -e cloud -noprompt) + * + *

You can run this test by doing the following from the beam parent module directory: + * + *

+ *  mvn -e -Pio-it verify -pl sdks/java/io/solr -DintegrationTestPipelineOptions='[
+ *  "--zookeeperSolrServer=127.0.0.1:9983"]'
+ * 
+ */ +public class SolrIOIT { + + public static final String READ_COLLECTION = "beam"; + public static final int NUM_SHARDS = 3; + public static final long NUM_DOCS = 60000; + public static final String WRITE_COLLECTION = "beam" + Instant.now().getMillis(); + private static final Logger LOG = LoggerFactory.getLogger(SolrIOIT.class); + + private static CloudSolrClient solrClient; + private static IOTestPipelineOptions options; + private static SolrIO.ConnectionConfiguration readConnectionConfiguration; + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void beforeClass() throws Exception { + PipelineOptionsFactory.register(IOTestPipelineOptions.class); + options = TestPipeline.testingPipelineOptions().as(IOTestPipelineOptions.class); + readConnectionConfiguration = SolrIO.ConnectionConfiguration + .create(options.getZookeeperSolrServer(), READ_COLLECTION); + solrClient = readConnectionConfiguration.createClient(); + List collections = CollectionAdminRequest.listCollections(solrClient); + if (collections.contains(READ_COLLECTION)) { + QueryResponse response = solrClient.query( + READ_COLLECTION, new SolrQuery("*:*")); + if (response.getResults().getNumFound() != NUM_DOCS) { + LOG.info("Collection {} is exist but the number of documents is not match, repopulate", + READ_COLLECTION); + SolrIOTestUtils.clearCollection(READ_COLLECTION, solrClient); + SolrIOTestUtils.insertTestDocuments(READ_COLLECTION, NUM_DOCS, solrClient); + } + } else { + LOG.info("Create and populate collection {}", + READ_COLLECTION); + } + + CollectionAdminRequest.createCollection(WRITE_COLLECTION, 2, 2) + .setMaxShardsPerNode(2).process(solrClient); + } + + @AfterClass + public static void afterClass() throws Exception { + SolrIOTestUtils.deleteCollection(WRITE_COLLECTION, solrClient); + solrClient.close(); + } + + @Test + public void testSplitsVolume() throws Exception { + SolrIO.Read read = + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.BoundedSolrSource initialSource = + new SolrIO.BoundedSolrSource(read, null); + //desiredBundleSize is ignored now + long desiredBundleSizeBytes = 0; + List> splits = + initialSource.split(desiredBundleSizeBytes, options); + SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, options); + //this is the number of Solr shards + long expectedNumSplits = NUM_SHARDS; + assertEquals(expectedNumSplits, splits.size()); + int nonEmptySplits = 0; + for (BoundedSource subSource : splits) { + if (readFromSource(subSource, options).size() > 0) { + nonEmptySplits += 1; + } + } + assertEquals(expectedNumSplits, nonEmptySplits); + } + + @Test + public void testReadVolume() throws Exception { + PCollection output = + pipeline.apply( + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration)); + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(NUM_DOCS); + pipeline.run(); + } + + @Test + public void testWriteVolume() throws Exception { + SolrIO.ConnectionConfiguration writeConnectionConfiguration = + SolrIO.ConnectionConfiguration.create(options.getZookeeperSolrServer(), WRITE_COLLECTION); + List data = SolrIOTestUtils.createDocuments(NUM_DOCS); + pipeline + .apply(Create.of(data)) + .apply(SolrIO.write().withConnectionConfiguration(writeConnectionConfiguration)); + pipeline.run(); + + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs( + WRITE_COLLECTION, solrClient); + assertEquals(NUM_DOCS, currentNumDocs); + } + + @Test + public void testEstimatedSizesVolume() throws Exception { + SolrIO.Read read = + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.BoundedSolrSource initialSource = + new SolrIO.BoundedSolrSource(read, null); + // can't use equal assert as Solr collections never have same size + // (due to internal Lucene implementation) + long estimatedSize = initialSource.getEstimatedSizeBytes(options); + LOG.info("Estimated size: {}", estimatedSize); + assertThat( + "Wrong estimated size bellow minimum", + estimatedSize, + greaterThan(SolrIOTestUtils.AVERAGE_DOC_SIZE * NUM_DOCS)); + assertThat( + "Wrong estimated size beyond maximum", + estimatedSize, + greaterThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java new file mode 100644 index 000000000000..81fe82ad3718 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solr; + +import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; +import static org.hamcrest.Matchers.greaterThan; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import java.util.List; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SourceTestUtils; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.PCollection; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.cloud.SolrCloudTestCase; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A test of {@link SolrIO} on an independent Solr instance. + */ +@ThreadLeakFilters(defaultFilters = true, filters = { + BeamThreadsFilter.class +}) +public class SolrIOTest extends SolrCloudTestCase{ + private static final Logger LOG = LoggerFactory.getLogger(SolrIOTest.class); + + private static final String SOLR_COLLECTION = "beam"; + public static final int NUM_SHARDS = 3; + private static final long NUM_DOCS = 400L; + private static final int NUM_SCIENTISTS = 10; + private static final long BATCH_SIZE = 200L; + + private static CloudSolrClient solrClient; + private static SolrIO.ConnectionConfiguration connectionConfiguration; + + @Rule + public TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void beforeClass() throws Exception { + configureCluster(3) + .addConfig("conf", getFile("cloud-minimal/conf").toPath()) + .configure(); + + + SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, cluster.getSolrClient()); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, cluster.getSolrClient()); + + String zkAddress = cluster.getZkServer().getZkAddress(); + connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION); + solrClient = connectionConfiguration.createClient(); + } + + @AfterClass + public static void afterClass() throws Exception { + solrClient.close(); + } + + @Before + public void before() throws Exception { + SolrIOTestUtils.clearCollection(SOLR_COLLECTION, solrClient); + } + + @Test + public void testSizes() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PipelineOptions options = PipelineOptionsFactory.create(); + SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); + // can't use equal assert as Solr collections never have same size + // (due to internal Lucene implementation) + long estimatedSize = initialSource.getEstimatedSizeBytes(options); + LOG.info("Estimated size: {}", estimatedSize); + assertThat( + "Wrong estimated size bellow minimum", + estimatedSize, + greaterThan(SolrIOTestUtils.AVERAGE_DOC_SIZE * NUM_DOCS)); + assertThat( + "Wrong estimated size beyond maximum", + estimatedSize, + greaterThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); + } + + @Test + public void testRead() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PCollection output = + pipeline.apply( + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .withBatchSize(100L)); + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(NUM_DOCS); + pipeline.run(); + } + + @Test + public void testReadWithQuery() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PCollection output = + pipeline.apply( + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .withQuery("scientist:Einstein")); + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); + pipeline.run(); + } + + @Test + public void testWrite() throws Exception { + List data = SolrIOTestUtils.createDocuments(NUM_DOCS); + pipeline + .apply(Create.of(data)) + .apply(SolrIO.write().withConnectionConfiguration(connectionConfiguration)); + pipeline.run(); + + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + assertEquals(NUM_DOCS, currentNumDocs); + + QueryResponse response = solrClient.query(new SolrQuery("scientist:Einstein")); + assertEquals(NUM_DOCS / NUM_SCIENTISTS, response.getResults().getNumFound()); + } + + @Test + public void testWriteWithMaxBatchSize() throws Exception { + SolrIO.Write write = + SolrIO.write() + .withConnectionConfiguration(connectionConfiguration) + .withMaxBatchSize(BATCH_SIZE); + // write bundles size is the runner decision, we cannot force a bundle size, + // so we test the Writer as a DoFn outside of a runner. + DoFnTester fnTester = DoFnTester.of(new SolrIO.Write.WriteFn(write)); + List input = SolrIOTestUtils.createDocuments(NUM_DOCS); + long numDocsProcessed = 0; + long numDocsInserted = 0; + for (SolrInputDocument document : input) { + fnTester.processElement(document); + numDocsProcessed++; + // test every 100 docs to avoid overloading Solr + if ((numDocsProcessed % 100) == 0) { + // force the index to upgrade after inserting for the inserted docs + // to be searchable immediately + long currentNumDocs = SolrIOTestUtils + .commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + if ((numDocsProcessed % BATCH_SIZE) == 0) { + /* bundle end */ + assertEquals( + "we are at the end of a bundle, we should have inserted all processed documents", + numDocsProcessed, + currentNumDocs); + numDocsInserted = currentNumDocs; + } else { + /* not bundle end */ + assertEquals( + "we are not at the end of a bundle, we should have inserted no more documents", + numDocsInserted, + currentNumDocs); + } + + } + } + fnTester.finishBundle(); + fnTester.close(); + } + + @Test + public void testSplit() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PipelineOptions options = PipelineOptionsFactory.create(); + SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); + //desiredBundleSize is ignored for now + int desiredBundleSizeBytes = 0; + List> splits = + initialSource.split(desiredBundleSizeBytes, options); + SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, options); + + int expectedNumSplits = NUM_SHARDS; + assertEquals(expectedNumSplits, splits.size()); + int nonEmptySplits = 0; + for (BoundedSource subSource : splits) { + if (readFromSource(subSource, options).size() > 0) { + nonEmptySplits += 1; + } + } + assertEquals("Wrong number of empty splits", expectedNumSplits, nonEmptySplits); + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java new file mode 100644 index 000000000000..3316e2b09e36 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -0,0 +1,107 @@ +package org.apache.beam.sdk.io.solr; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +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.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.common.SolrInputDocument; + + +/** Test utilities to use with {@link SolrIO}. */ +public class SolrIOTestUtils { + public static final long AVERAGE_DOC_SIZE = 25L; + public static final long MAX_DOC_SIZE = 35L; + + static void createCollection(String collection, int numShards, SolrClient client) + throws Exception { + CollectionAdminRequest.createCollection(collection, numShards, 1) + .setMaxShardsPerNode(2) + .process(client); + } + + /** Inserts the given number of test documents into Solr. */ + static void insertTestDocuments(String collection, long numDocs, CloudSolrClient client) + throws IOException { + List data = createDocuments(numDocs); + try { + client.add(collection, data); + client.commit(collection); + } catch (SolrServerException e) { + throw new IOException("Failed to insert test documents in collection " + collection, e); + } + } + + + /** Delete given collection. */ + static void deleteCollection(String collection, SolrClient client) + throws IOException { + try { + CollectionAdminRequest.deleteCollection(collection).process(client); + } catch (SolrServerException e) { + throw new IOException(e); + } + + } + + /** Clear given collection. */ + static void clearCollection(String collection, SolrClient client) throws IOException { + try { + client.deleteByQuery(collection, "*:*"); + client.commit(); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + /** + * Forces a commit of the given collection to + * make recently inserted documents available for search. + * + * @return The number of docs in the index + */ + static long commitAndGetCurrentNumDocs(String collection, SolrClient client) + throws IOException { + SolrQuery solrQuery = new SolrQuery("*:*"); + solrQuery.setRows(0); + try { + client.commit(collection); + return client.query(solrQuery).getResults().getNumFound(); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + /** + * Generates a list of test documents for insertion. + * + * @return the list of json String representing the documents + */ + static List createDocuments(long numDocs) { + String[] scientists = { + "Einstein", + "Darwin", + "Copernicus", + "Pasteur", + "Curie", + "Faraday", + "Newton", + "Bohr", + "Galilei", + "Maxwell" + }; + ArrayList data = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + int index = i % scientists.length; + SolrInputDocument doc = new SolrInputDocument( + "scientist", scientists[index], "id", String.valueOf(i)); + data.add(doc); + } + return data; + } + + +} diff --git a/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml new file mode 100644 index 000000000000..08a1716deb14 --- /dev/null +++ b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + id + diff --git a/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml new file mode 100644 index 000000000000..8da7d2847e9b --- /dev/null +++ b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml @@ -0,0 +1,48 @@ + + + + + + + + + ${solr.data.dir:} + + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + + explicit + true + text + + + + + From 6cf1777d22f93e4922330ff9e97e9f76ebe14c8f Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Sat, 22 Jul 2017 15:23:40 +0700 Subject: [PATCH 02/14] Add Apache license header for SolrIO --- sdks/java/io/solr/pom.xml | 16 +++++++++++++++ .../org/apache/beam/sdk/io/solr/SolrIO.java | 18 ++++++++++++++++- .../beam/sdk/io/solr/BeamThreadsFilter.java | 17 ++++++++++++++++ .../beam/sdk/io/solr/SolrIOCoderTest.java | 20 +++++++++++++++++-- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 1 - .../beam/sdk/io/solr/SolrIOTestUtils.java | 17 ++++++++++++++++ 6 files changed, 85 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index 0fd2fa9a3774..cf7f2bb96313 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -1,4 +1,20 @@ + diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 518ca272b109..b5e2d970edae 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -1,6 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.solr; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java index c94c2ff71429..ebcc6c608d1d 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.solr; import com.carrotsearch.randomizedtesting.ThreadFilter; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java index fae67e0fdba5..5a7bc0be4faf 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java @@ -1,15 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.beam.sdk.io.solr; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; import com.google.common.io.CountingInputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.UnownedInputStream; import org.apache.beam.sdk.util.UnownedOutputStream; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index 7b03994598f5..db3970128738 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals; import java.util.List; - import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java index 3316e2b09e36..a5e71c75c152 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.solr; import java.io.IOException; From 5f96b959834b08e9d2c779e0cb355d05c5ae7d2e Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Sat, 22 Jul 2017 21:18:25 +0700 Subject: [PATCH 03/14] Fix problem because of org.apache.solr.common.cloud.Replica is not serializable --- .../org/apache/beam/sdk/io/solr/SolrIO.java | 32 +++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index b5e2d970edae..611115d1c30a 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -38,6 +38,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; @@ -193,7 +194,7 @@ HttpSolrClient createClient(String shardUrl) { private static final long MAX_BATCH_SIZE = 10000L; - @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); + abstract ConnectionConfiguration getConnectionConfiguration(); @Nullable abstract String getQuery(); @@ -277,16 +278,29 @@ public Read withBatchSize(long batchSize) { } } + /** A POJO describing a replica of Solr. */ + @AutoValue + abstract static class ReplicaInfo implements Serializable { + public abstract String coreName(); + public abstract String coreUrl(); + public abstract String baseUrl(); + + static ReplicaInfo create(Replica replica) { + return new AutoValue_SolrIO_ReplicaInfo(replica.getCoreName(), + replica.getCoreUrl(), replica.getBaseUrl()); + } + } + /** A {@link BoundedSource} reading from Solr. */ @VisibleForTesting static class BoundedSolrSource extends BoundedSource { private final SolrIO.Read spec; // replica is the info of the shard where the source will read the documents - @Nullable private final Replica replica; + @Nullable private final ReplicaInfo replica; BoundedSolrSource(Read spec, @Nullable Replica replica) { this.spec = spec; - this.replica = replica; + this.replica = replica == null ? null : ReplicaInfo.create(replica); } @Override public List> split(long desiredBundleSizeBytes, @@ -316,9 +330,9 @@ public Read withBatchSize(long batchSize) { } } - private long getEstimatedSizeOfShard(Replica replica) throws IOException { + private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOException { try (HttpSolrClient solrClient = spec.getConnectionConfiguration() - .createClient(replica.getBaseUrl())) { + .createClient(replica.baseUrl())) { CoreAdminRequest req = new CoreAdminRequest(); req.setAction(CoreAdminParams.CoreAdminAction.STATUS); req.setIndexInfoNeeded(true); @@ -328,7 +342,7 @@ private long getEstimatedSizeOfShard(Replica replica) throws IOException { } catch (SolrServerException e) { throw new IOException("Can not get core status from " + replica, e); } - NamedList coreStatus = response.getCoreStatus(replica.getCoreName()); + NamedList coreStatus = response.getCoreStatus(replica.coreName()); NamedList indexStats = (NamedList) coreStatus.get("index"); return (long) indexStats.get("sizeInBytes"); } @@ -343,7 +357,7 @@ private long getEstimatedSizeOfCollection() throws IOException { DocCollection docCollection = clusterState.getCollection(config.getCollection()); for (Slice slice : docCollection.getSlices()) { Replica replica = slice.getLeader(); - sizeInBytes += getEstimatedSizeOfShard(replica); + sizeInBytes += getEstimatedSizeOfShard(ReplicaInfo.create(replica)); } } return sizeInBytes; @@ -352,7 +366,7 @@ private long getEstimatedSizeOfCollection() throws IOException { @Override public void populateDisplayData(DisplayData.Builder builder) { spec.populateDisplayData(builder); if (replica != null) { - builder.addIfNotNull(DisplayData.item("shardUrl", replica.getCoreUrl())); + builder.addIfNotNull(DisplayData.item("shardUrl", replica.coreUrl())); } } @@ -434,7 +448,7 @@ private BoundedSolrReader(BoundedSolrSource source) { @Override public boolean start() throws IOException { if (source.replica != null) { solrClient = source.spec.getConnectionConfiguration() - .createClient(source.replica.getCoreUrl()); + .createClient(source.replica.coreUrl()); } else { solrClient = source.spec.getConnectionConfiguration().createClient(); } From fc4af7a926353a8369cab450d6e377804d88b244 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 28 Jul 2017 10:37:27 +0700 Subject: [PATCH 04/14] Adding basic authentication support for SolrIO --- .../io/solr/AuthorizedCloudSolrClient.java | 40 ++++ .../sdk/io/solr/AuthorizedSolrClient.java | 55 ++++++ .../org/apache/beam/sdk/io/solr/SolrIO.java | 172 ++++++++++++------ .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 23 ++- .../apache/beam/sdk/io/solr/SolrIOTest.java | 50 +++-- .../beam/sdk/io/solr/SolrIOTestUtils.java | 43 +++-- 6 files changed, 281 insertions(+), 102 deletions(-) create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java new file mode 100644 index 000000000000..b78bfdcd2aa0 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java @@ -0,0 +1,40 @@ +package org.apache.beam.sdk.io.solr; + +import java.io.IOException; + +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.SolrResponse; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.cloud.DocCollection; +import org.apache.solr.common.params.SolrParams; + +/** + * Client for interact with SolrCloud. + */ +public class AuthorizedCloudSolrClient extends AuthorizedSolrClient { + + public AuthorizedCloudSolrClient(CloudSolrClient solrClient, + SolrIO.ConnectionConfiguration configuration) { + super(solrClient, configuration); + } + + public DocCollection getDocCollection(String collection){ + solrClient.connect(); + return solrClient.getZkStateReader().getClusterState().getCollection(collection); + } + + public QueryResponse query(String collection, SolrParams solrParams) + throws IOException, SolrServerException { + QueryRequest query = new QueryRequest(solrParams); + return process(collection, query); + } + + public T process(String collection, SolrRequest request) + throws IOException, SolrServerException { + request.setBasicAuthCredentials(username, password); + return request.process(solrClient, collection); + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java new file mode 100644 index 000000000000..81e4e7d1eb3a --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java @@ -0,0 +1,55 @@ +package org.apache.beam.sdk.io.solr; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.beam.sdk.io.solr.SolrIO.ConnectionConfiguration; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.SolrResponse; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.params.SolrParams; + +/** + * Client for interact with a replica in Solr. + * @param type of SolrClient + */ +class AuthorizedSolrClient implements Closeable { + protected T solrClient; + protected String username; + protected String password; + + public AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) { + checkArgument( + solrClient != null, + "AuthorizedSolrClient(solrClient, configuration) " + + "called with null solrClient"); + checkArgument( + configuration != null, + "AuthorizedSolrClient(solrClient, configuration) " + + "called with null configuration"); + this.solrClient = solrClient; + this.username = configuration.getUsername(); + this.password = configuration.getPassword(); + } + + public QueryResponse query(SolrParams solrParams) + throws IOException, SolrServerException { + QueryRequest query = new QueryRequest(solrParams); + return process(query); + } + + public T process(SolrRequest request) + throws IOException, SolrServerException { + request.setBasicAuthCredentials(username, password); + return request.process(solrClient); + } + + @Override public void close() throws IOException { + solrClient.close(); + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 611115d1c30a..bed845b6ec21 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -54,22 +54,24 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.solr.client.solrj.SolrClient; +import org.apache.http.client.HttpClient; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.impl.HttpClientUtil; import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.CoreAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.client.solrj.response.CoreAdminResponse; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrInputDocument; -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.params.CoreAdminParams; import org.apache.solr.common.params.CursorMarkParams; +import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.JavaBinCodec; import org.apache.solr.common.util.NamedList; @@ -120,12 +122,37 @@ @Experimental public class SolrIO { - public static Read read() { - return new AutoValue_SolrIO_Read.Builder().setBatchSize(100L).build(); + /** + * Create new SolrIO.Read based on provided Solr connection configuration object. + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Read} with connection configuration set + */ + public static Read read(ConnectionConfiguration connectionConfiguration) { + checkArgument( + connectionConfiguration != null, + "SolrIO.read(connectionConfiguration) " + + "called with null connectionConfiguration"); + return new AutoValue_SolrIO_Read.Builder() + .setConnectionConfiguration(connectionConfiguration) + .setBatchSize(100L) + .setQuery("*:*") + .build(); } - public static Write write() { - return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000L).build(); + /** + * Create new SolrIO.Write based on provided Solr connection configuration object. + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Write} with connection configuration set + */ + public static Write write(ConnectionConfiguration connectionConfiguration) { + checkArgument( + connectionConfiguration != null, + "SolrIO.write(connectionConfiguration) " + + "called with null connectionConfiguration"); + return new AutoValue_SolrIO_Write.Builder() + .setMaxBatchSize(1000L) + .setConnectionConfiguration(connectionConfiguration) + .build(); } private SolrIO() { @@ -134,9 +161,14 @@ private SolrIO() { /** A POJO describing a connection configuration to Solr. */ @AutoValue public abstract static class ConnectionConfiguration implements Serializable { - //TODO add user name password abstract String getZkHost(); + @Nullable + abstract String getUsername(); + + @Nullable + abstract String getPassword(); + abstract String getCollection(); abstract Builder builder(); @@ -145,6 +177,10 @@ private SolrIO() { abstract Builder setZkHost(String zkHost); + abstract Builder setUsername(String username); + + abstract Builder setPassword(String password); + abstract Builder setCollection(String collection); abstract ConnectionConfiguration build(); @@ -166,25 +202,70 @@ public static ConnectionConfiguration create(String zkHost, String collection) checkArgument(collection != null, "ConnectionConfiguration.create(zkHost, collection) " + "called with null collectioin"); - ConnectionConfiguration connectionConfiguration = new AutoValue_SolrIO_ConnectionConfiguration - .Builder().setZkHost(zkHost).setCollection(collection).build(); - return connectionConfiguration; + ConnectionConfiguration connectionConfig = new AutoValue_SolrIO_ConnectionConfiguration + .Builder() + .setZkHost(zkHost) + .setCollection(collection) + .build(); + return connectionConfig; + } + + /** + * If Solr basic authentication is enabled, provide the username and password. + * + * @param username the username used to authenticate to Solr + * @param password the password used to authenticate to Solr + * @return the {@link ConnectionConfiguration} object with basic credentials set + */ + public ConnectionConfiguration withBasicCredentials(String username, String password) { + checkArgument( + username != null, + "ConnectionConfiguration.create().withBasicCredentials(username, password) " + + "called with null username"); + checkArgument( + !username.isEmpty(), + "ConnectionConfiguration.create().withBasicCredentials(username, password) " + + "called with empty username"); + checkArgument( + password != null, + "ConnectionConfiguration.create().withBasicCredentials(username, password) " + + "called with null username"); + checkArgument( + !password.isEmpty(), + "ConnectionConfiguration.create().withBasicCredentials(username, password) " + + "called with empty username"); + return builder().setUsername(username).setPassword(password).build(); } private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("zkHost", getZkHost())); builder.add(DisplayData.item("collection", getCollection())); + builder.addIfNotNull(DisplayData.item("username", getUsername())); } - CloudSolrClient createClient() throws MalformedURLException { - CloudSolrClient solrClient = new CloudSolrClient.Builder().withZkHost(getZkHost()).build(); + private HttpClient createHttpClient() { + // This is bug in Solr, if we don't create a customize HttpClient, + // UpdateRequest with commit flag will throw an authentication error. + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set(HttpClientUtil.PROP_BASIC_AUTH_USER, getUsername()); + params.set(HttpClientUtil.PROP_BASIC_AUTH_PASS, getPassword()); + return HttpClientUtil.createClient(params); + } + + AuthorizedCloudSolrClient createClient() throws MalformedURLException { + CloudSolrClient solrClient = new CloudSolrClient.Builder() + .withHttpClient(createHttpClient()) + .withZkHost(getZkHost()) + .build(); solrClient.setDefaultCollection(getCollection()); - return solrClient; + return new AuthorizedCloudSolrClient(solrClient, this); } - HttpSolrClient createClient(String shardUrl) { - HttpSolrClient solrClient = new HttpSolrClient.Builder(shardUrl).build(); - return solrClient; + AuthorizedSolrClient createClient(String shardUrl) { + HttpSolrClient solrClient = new HttpSolrClient.Builder(shardUrl) + .withHttpClient(createHttpClient()) + .build(); + return new AuthorizedSolrClient<>(solrClient, this); } } @@ -196,7 +277,7 @@ HttpSolrClient createClient(String shardUrl) { abstract ConnectionConfiguration getConnectionConfiguration(); - @Nullable abstract String getQuery(); + abstract String getQuery(); abstract long getBatchSize(); @@ -213,18 +294,6 @@ HttpSolrClient createClient(String shardUrl) { abstract Read build(); } - /** - * Provide the Solr connection configuration object. - * - * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Read} with connection configuration set - */ - public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { - checkArgument(connectionConfiguration != null, "SolrIO.read()" - + ".withConnectionConfiguration(configuration) called with null configuration"); - return builder().setConnectionConfiguration(connectionConfiguration).build(); - } - /** * Provide a query used while reading from Solr. * @@ -307,12 +376,9 @@ static ReplicaInfo create(Replica replica) { PipelineOptions options) throws Exception { List sources = new ArrayList<>(); int numShard; - try (CloudSolrClient client = spec.getConnectionConfiguration().createClient()) { - // connect to zk cluster - client.connect(); - ClusterState clusterState = client.getZkStateReader().getClusterState(); - DocCollection docCollection = clusterState - .getCollection(spec.getConnectionConfiguration().getCollection()); + try (AuthorizedCloudSolrClient client = spec.getConnectionConfiguration().createClient()) { + String collection = spec.getConnectionConfiguration().getCollection(); + DocCollection docCollection = client.getDocCollection(collection); numShard = docCollection.getSlices().size(); for (Slice slice : docCollection.getSlices()) { sources.add(new BoundedSolrSource(spec, slice.getLeader())); @@ -331,14 +397,14 @@ static ReplicaInfo create(Replica replica) { } private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOException { - try (HttpSolrClient solrClient = spec.getConnectionConfiguration() + try (AuthorizedSolrClient solrClient = spec.getConnectionConfiguration() .createClient(replica.baseUrl())) { CoreAdminRequest req = new CoreAdminRequest(); req.setAction(CoreAdminParams.CoreAdminAction.STATUS); req.setIndexInfoNeeded(true); CoreAdminResponse response; try { - response = req.process(solrClient); + response = (CoreAdminResponse) solrClient.process(req); } catch (SolrServerException e) { throw new IOException("Can not get core status from " + replica, e); } @@ -351,10 +417,8 @@ private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOExce private long getEstimatedSizeOfCollection() throws IOException { long sizeInBytes = 0; ConnectionConfiguration config = spec.getConnectionConfiguration(); - try (CloudSolrClient solrClient = config.createClient()) { - solrClient.connect(); - ClusterState clusterState = solrClient.getZkStateReader().getClusterState(); - DocCollection docCollection = clusterState.getCollection(config.getCollection()); + try (AuthorizedCloudSolrClient solrClient = config.createClient()) { + DocCollection docCollection = solrClient.getDocCollection(config.getCollection()); for (Slice slice : docCollection.getSlices()) { Replica replica = slice.getLeader(); sizeInBytes += getEstimatedSizeOfShard(ReplicaInfo.create(replica)); @@ -434,7 +498,7 @@ private static class BoundedSolrReader extends BoundedSource.BoundedReader batchIterator; @@ -450,7 +514,8 @@ private BoundedSolrReader(BoundedSolrSource source) { solrClient = source.spec.getConnectionConfiguration() .createClient(source.replica.coreUrl()); } else { - solrClient = source.spec.getConnectionConfiguration().createClient(); + solrClient = source.spec.getConnectionConfiguration() + .createClient(); } SolrQuery solrParams = getQueryParams(source); @@ -532,7 +597,7 @@ private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { @AutoValue public abstract static class Write extends PTransform, PDone> { - @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); + abstract ConnectionConfiguration getConnectionConfiguration(); abstract long getMaxBatchSize(); @@ -547,18 +612,6 @@ private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { abstract Write build(); } - /** - * Provide the Solr connection configuration object. - * - * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Write} with connection configuration set - */ - public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { - checkArgument(connectionConfiguration != null, "SolrIO.write()" - + ".withConnectionConfiguration(configuration) called with null configuration"); - return builder().setConnectionConfiguration(connectionConfiguration).build(); - } - /** * Provide a maximum size in number of documents for the batch. Depending on the * execution engine, size of bundles may vary, this sets the maximum size. Change this if you @@ -588,7 +641,7 @@ public Write withMaxBatchSize(long batchSize) { private final Write spec; - private transient CloudSolrClient solrClient; + private transient AuthorizedSolrClient solrClient; private Collection batch; WriteFn(Write spec) { @@ -597,7 +650,6 @@ public Write withMaxBatchSize(long batchSize) { @Setup public void createClient() throws Exception { solrClient = spec.getConnectionConfiguration().createClient(); - solrClient.connect(); } @StartBundle public void startBundle(StartBundleContext context) throws Exception { @@ -622,7 +674,9 @@ private void flushBatch() throws IOException { return; } try { - solrClient.add(spec.getConnectionConfiguration().getCollection(), batch); + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.add(batch); + solrClient.process(updateRequest); } catch (SolrServerException e) { throw new IOException("Error writing to Solr", e); } finally { diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index db3970128738..4cad8d2adb8b 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals; import java.util.List; + import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -33,8 +34,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.apache.solr.client.solrj.SolrQuery; -import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrInputDocument; @@ -66,7 +67,7 @@ public class SolrIOIT { public static final String WRITE_COLLECTION = "beam" + Instant.now().getMillis(); private static final Logger LOG = LoggerFactory.getLogger(SolrIOIT.class); - private static CloudSolrClient solrClient; + private static AuthorizedCloudSolrClient solrClient; private static IOTestPipelineOptions options; private static SolrIO.ConnectionConfiguration readConnectionConfiguration; @Rule public TestPipeline pipeline = TestPipeline.create(); @@ -78,7 +79,8 @@ public static void beforeClass() throws Exception { readConnectionConfiguration = SolrIO.ConnectionConfiguration .create(options.getZookeeperSolrServer(), READ_COLLECTION); solrClient = readConnectionConfiguration.createClient(); - List collections = CollectionAdminRequest.listCollections(solrClient); + CollectionAdminResponse listResponse = solrClient.process(new CollectionAdminRequest.List()); + List collections = (List) listResponse.getResponse().get("collections"); if (collections.contains(READ_COLLECTION)) { QueryResponse response = solrClient.query( READ_COLLECTION, new SolrQuery("*:*")); @@ -93,8 +95,11 @@ public static void beforeClass() throws Exception { READ_COLLECTION); } - CollectionAdminRequest.createCollection(WRITE_COLLECTION, 2, 2) - .setMaxShardsPerNode(2).process(solrClient); + CollectionAdminRequest.Create create = CollectionAdminRequest + .createCollection(WRITE_COLLECTION, 2, 2) + .setMaxShardsPerNode(2); + solrClient.process(create); + } @AfterClass @@ -106,7 +111,7 @@ public static void afterClass() throws Exception { @Test public void testSplitsVolume() throws Exception { SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.read(readConnectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored now @@ -130,7 +135,7 @@ public void testSplitsVolume() throws Exception { public void testReadVolume() throws Exception { PCollection output = pipeline.apply( - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration)); + SolrIO.read(readConnectionConfiguration)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); pipeline.run(); @@ -143,7 +148,7 @@ public void testWriteVolume() throws Exception { List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) - .apply(SolrIO.write().withConnectionConfiguration(writeConnectionConfiguration)); + .apply(SolrIO.write(writeConnectionConfiguration)); pipeline.run(); long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs( @@ -154,7 +159,7 @@ public void testWriteVolume() throws Exception { @Test public void testEstimatedSizesVolume() throws Exception { SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.read(readConnectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 81fe82ad3718..860641a9d0dc 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -22,6 +22,7 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import java.io.IOException; import java.util.List; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -34,10 +35,10 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.PCollection; import org.apache.solr.client.solrj.SolrQuery; -import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.junit.AfterClass; import org.junit.Before; @@ -62,7 +63,7 @@ public class SolrIOTest extends SolrCloudTestCase{ private static final int NUM_SCIENTISTS = 10; private static final long BATCH_SIZE = 200L; - private static CloudSolrClient solrClient; + private static AuthorizedCloudSolrClient solrClient; private static SolrIO.ConnectionConfiguration connectionConfiguration; @Rule @@ -70,17 +71,24 @@ public class SolrIOTest extends SolrCloudTestCase{ @BeforeClass public static void beforeClass() throws Exception { + String securityJson = "{" + + "'authentication':{" + + " 'blockUnknown': true," + + " 'class':'solr.BasicAuthPlugin'," + + " 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=" + + " Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}}" + + "}"; configureCluster(3) .addConfig("conf", getFile("cloud-minimal/conf").toPath()) + .withSecurityJson(securityJson) .configure(); - - SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, cluster.getSolrClient()); - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, cluster.getSolrClient()); - String zkAddress = cluster.getZkServer().getZkAddress(); - connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION); + connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION) + .withBasicCredentials("solr", "SolrRocks"); solrClient = connectionConfiguration.createClient(); + SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); } @AfterClass @@ -93,12 +101,23 @@ public void before() throws Exception { SolrIOTestUtils.clearCollection(SOLR_COLLECTION, solrClient); } + @Test(expected = SolrException.class) + public void testBadCredentials() throws IOException { + String zkAddress = cluster.getZkServer().getZkAddress(); + SolrIO.ConnectionConfiguration connectionConfiguration = SolrIO.ConnectionConfiguration + .create(zkAddress, SOLR_COLLECTION) + .withBasicCredentials("solr", "wrongpassword"); + try (AuthorizedCloudSolrClient solrClient = connectionConfiguration.createClient()) { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + } + } + @Test public void testSizes() throws Exception { SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.Read read = SolrIO.read(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size // (due to internal Lucene implementation) @@ -120,8 +139,7 @@ public void testRead() throws Exception { PCollection output = pipeline.apply( - SolrIO.read() - .withConnectionConfiguration(connectionConfiguration) + SolrIO.read(connectionConfiguration) .withBatchSize(100L)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); @@ -134,8 +152,7 @@ public void testReadWithQuery() throws Exception { PCollection output = pipeline.apply( - SolrIO.read() - .withConnectionConfiguration(connectionConfiguration) + SolrIO.read(connectionConfiguration) .withQuery("scientist:Einstein")); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); @@ -147,21 +164,20 @@ public void testWrite() throws Exception { List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) - .apply(SolrIO.write().withConnectionConfiguration(connectionConfiguration)); + .apply(SolrIO.write(connectionConfiguration)); pipeline.run(); long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); assertEquals(NUM_DOCS, currentNumDocs); - QueryResponse response = solrClient.query(new SolrQuery("scientist:Einstein")); + QueryResponse response = solrClient.query(SOLR_COLLECTION, new SolrQuery("scientist:Einstein")); assertEquals(NUM_DOCS / NUM_SCIENTISTS, response.getResults().getNumFound()); } @Test public void testWriteWithMaxBatchSize() throws Exception { SolrIO.Write write = - SolrIO.write() - .withConnectionConfiguration(connectionConfiguration) + SolrIO.write(connectionConfiguration) .withMaxBatchSize(BATCH_SIZE); // write bundles size is the runner decision, we cannot force a bundle size, // so we test the Writer as a DoFn outside of a runner. @@ -204,7 +220,7 @@ public void testSplit() throws Exception { SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.Read read = SolrIO.read(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored for now int desiredBundleSizeBytes = 0; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java index a5e71c75c152..d2fc620feb05 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -20,11 +20,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -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.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.common.SolrInputDocument; @@ -33,20 +32,23 @@ public class SolrIOTestUtils { public static final long AVERAGE_DOC_SIZE = 25L; public static final long MAX_DOC_SIZE = 35L; - static void createCollection(String collection, int numShards, SolrClient client) + static void createCollection(String collection, int numShards, AuthorizedCloudSolrClient client) throws Exception { - CollectionAdminRequest.createCollection(collection, numShards, 1) - .setMaxShardsPerNode(2) - .process(client); + CollectionAdminRequest.Create create = CollectionAdminRequest + .createCollection(collection, numShards, 1) + .setMaxShardsPerNode(2); + client.process(create); } /** Inserts the given number of test documents into Solr. */ - static void insertTestDocuments(String collection, long numDocs, CloudSolrClient client) + static void insertTestDocuments(String collection, long numDocs, AuthorizedCloudSolrClient client) throws IOException { List data = createDocuments(numDocs); try { - client.add(collection, data); - client.commit(collection); + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); + updateRequest.add(data); + client.process(collection, updateRequest); } catch (SolrServerException e) { throw new IOException("Failed to insert test documents in collection " + collection, e); } @@ -54,10 +56,11 @@ static void insertTestDocuments(String collection, long numDocs, CloudSolrClient /** Delete given collection. */ - static void deleteCollection(String collection, SolrClient client) + static void deleteCollection(String collection, AuthorizedCloudSolrClient client) throws IOException { try { - CollectionAdminRequest.deleteCollection(collection).process(client); + CollectionAdminRequest.Delete delete = CollectionAdminRequest.deleteCollection(collection); + client.process(delete); } catch (SolrServerException e) { throw new IOException(e); } @@ -65,10 +68,13 @@ static void deleteCollection(String collection, SolrClient client) } /** Clear given collection. */ - static void clearCollection(String collection, SolrClient client) throws IOException { + static void clearCollection(String collection, AuthorizedCloudSolrClient client) + throws IOException { try { - client.deleteByQuery(collection, "*:*"); - client.commit(); + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); + updateRequest.deleteByQuery("*:*"); + client.process(collection, updateRequest); } catch (SolrServerException e) { throw new IOException(e); } @@ -80,13 +86,16 @@ static void clearCollection(String collection, SolrClient client) throws IOExcep * * @return The number of docs in the index */ - static long commitAndGetCurrentNumDocs(String collection, SolrClient client) + static long commitAndGetCurrentNumDocs(String collection, AuthorizedCloudSolrClient client) throws IOException { SolrQuery solrQuery = new SolrQuery("*:*"); solrQuery.setRows(0); try { - client.commit(collection); - return client.query(solrQuery).getResults().getNumFound(); + UpdateRequest update = new UpdateRequest(); + update.setAction(UpdateRequest.ACTION.COMMIT, true, true); + client.process(collection, update); + + return client.query(new SolrQuery("*:*")).getResults().getNumFound(); } catch (SolrServerException e) { throw new IOException(e); } From e31724dd56c4fb6bc756bfbcce3a3aa5272a15a6 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 28 Jul 2017 12:14:24 +0700 Subject: [PATCH 05/14] Changing solr-solrj from 6.6.0 to 5.5.4 --- sdks/java/io/solr/pom.xml | 46 ++----------------- .../org/apache/beam/sdk/io/solr/SolrIO.java | 14 ++---- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 6 +-- .../apache/beam/sdk/io/solr/SolrIOTest.java | 23 ++++++++-- .../beam/sdk/io/solr/SolrIOTestUtils.java | 17 ++++--- 5 files changed, 40 insertions(+), 66 deletions(-) diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index cf7f2bb96313..143a8b44b072 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -30,46 +30,6 @@ Apache Beam :: SDKs :: Java :: IO :: Solr IO to read and write on Solr. - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4.1 - - - enforce - - enforce - - - - - 1.8 - - - jdk.tools:jdk.tools - - - - [1.8,) - - - - - - - - org.codehaus.mojo - extra-enforcer-rules - 1.0-beta-6 - - - - - - org.apache.beam @@ -84,7 +44,7 @@ org.apache.solr solr-solrj - 6.6.0 + 5.5.4 @@ -145,14 +105,14 @@ org.apache.solr solr-test-framework - 6.6.0 + 5.5.4 test com.carrotsearch.randomizedtesting randomizedtesting-runner - 2.5.0 + 2.3.2 test diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index bed845b6ec21..98793942bb06 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -69,6 +69,7 @@ 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.ZkStateReader; import org.apache.solr.common.params.CoreAdminParams; import org.apache.solr.common.params.CursorMarkParams; import org.apache.solr.common.params.ModifiableSolrParams; @@ -253,18 +254,13 @@ private HttpClient createHttpClient() { } AuthorizedCloudSolrClient createClient() throws MalformedURLException { - CloudSolrClient solrClient = new CloudSolrClient.Builder() - .withHttpClient(createHttpClient()) - .withZkHost(getZkHost()) - .build(); + CloudSolrClient solrClient = new CloudSolrClient(getZkHost(), createHttpClient()); solrClient.setDefaultCollection(getCollection()); return new AuthorizedCloudSolrClient(solrClient, this); } AuthorizedSolrClient createClient(String shardUrl) { - HttpSolrClient solrClient = new HttpSolrClient.Builder(shardUrl) - .withHttpClient(createHttpClient()) - .build(); + HttpSolrClient solrClient = new HttpSolrClient(shardUrl, createHttpClient()); return new AuthorizedSolrClient<>(solrClient, this); } } @@ -355,8 +351,8 @@ abstract static class ReplicaInfo implements Serializable { public abstract String baseUrl(); static ReplicaInfo create(Replica replica) { - return new AutoValue_SolrIO_ReplicaInfo(replica.getCoreName(), - replica.getCoreUrl(), replica.getBaseUrl()); + return new AutoValue_SolrIO_ReplicaInfo(replica.getStr(ZkStateReader.CORE_NAME_PROP), + replica.getCoreUrl(), replica.getStr(ZkStateReader.BASE_URL_PROP)); } } diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index 4cad8d2adb8b..64dc7a121c21 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -95,11 +95,7 @@ public static void beforeClass() throws Exception { READ_COLLECTION); } - CollectionAdminRequest.Create create = CollectionAdminRequest - .createCollection(WRITE_COLLECTION, 2, 2) - .setMaxShardsPerNode(2); - solrClient.process(create); - + SolrIOTestUtils.createCollection(WRITE_COLLECTION, 2, 2, solrClient); } @AfterClass diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 860641a9d0dc..5464e146c230 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -23,7 +23,10 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; import java.io.IOException; +import java.nio.charset.Charset; import java.util.List; +import java.util.concurrent.TimeUnit; + import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -40,6 +43,8 @@ import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.cloud.ZkStateReader; +import org.apache.solr.util.TimeOut; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -80,14 +85,26 @@ public static void beforeClass() throws Exception { + "}"; configureCluster(3) .addConfig("conf", getFile("cloud-minimal/conf").toPath()) - .withSecurityJson(securityJson) .configure(); - + ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader(); + zkStateReader.getZkClient() + .setData("/security.json", securityJson.getBytes(Charset.defaultCharset()), true); + TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS); + while (!timeOut.hasTimedOut()) { + if (zkStateReader.getClusterState().getLiveNodes().size() == 3) { + break; + } else { + Thread.sleep(100); + } + } + if (timeOut.hasTimedOut()) { + fail("Timeout waiting for nodes come back"); + } String zkAddress = cluster.getZkServer().getZkAddress(); connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION) .withBasicCredentials("solr", "SolrRocks"); solrClient = connectionConfiguration.createClient(); - SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, solrClient); + SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, 1, solrClient); SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); } diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java index d2fc620feb05..5582c2202c0b 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -32,10 +32,13 @@ public class SolrIOTestUtils { public static final long AVERAGE_DOC_SIZE = 25L; public static final long MAX_DOC_SIZE = 35L; - static void createCollection(String collection, int numShards, AuthorizedCloudSolrClient client) + static void createCollection(String collection, int numShards, int replicationFactor, + AuthorizedCloudSolrClient client) throws Exception { - CollectionAdminRequest.Create create = CollectionAdminRequest - .createCollection(collection, numShards, 1) + CollectionAdminRequest.Create create = new CollectionAdminRequest.Create() + .setCollectionName(collection) + .setNumShards(numShards) + .setReplicationFactor(replicationFactor) .setMaxShardsPerNode(2); client.process(create); } @@ -59,7 +62,8 @@ static void insertTestDocuments(String collection, long numDocs, AuthorizedCloud static void deleteCollection(String collection, AuthorizedCloudSolrClient client) throws IOException { try { - CollectionAdminRequest.Delete delete = CollectionAdminRequest.deleteCollection(collection); + CollectionAdminRequest.Delete delete = new CollectionAdminRequest.Delete() + .setCollectionName(collection); client.process(delete); } catch (SolrServerException e) { throw new IOException(e); @@ -122,8 +126,9 @@ static List createDocuments(long numDocs) { ArrayList data = new ArrayList<>(); for (int i = 0; i < numDocs; i++) { int index = i % scientists.length; - SolrInputDocument doc = new SolrInputDocument( - "scientist", scientists[index], "id", String.valueOf(i)); + SolrInputDocument doc = new SolrInputDocument(); + doc.setField("id", String.valueOf(i)); + doc.setField("scientist", scientists[index]); data.add(doc); } return data; From 0cf4fce3140157dc45abe18a7093b88bfbaad6c7 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 28 Jul 2017 22:38:58 +0700 Subject: [PATCH 06/14] Update based on reviews of @iemejia for SolrIO --- pom.xml | 8 +- .../sdk/io/common/IOTestPipelineOptions.java | 4 +- sdks/java/io/solr/pom.xml | 16 +- .../io/solr/AuthorizedCloudSolrClient.java | 27 ++- .../sdk/io/solr/AuthorizedSolrClient.java | 23 ++- .../beam/sdk/io/solr/SolrDocumentCoder.java | 73 ++++++++ .../org/apache/beam/sdk/io/solr/SolrIO.java | 176 ++++++------------ .../apache/beam/sdk/io/solr/package-info.java | 2 +- .../beam/sdk/io/solr/BeamThreadsFilter.java | 4 +- .../beam/sdk/io/solr/SolrIOCoderTest.java | 50 +++-- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 6 +- .../apache/beam/sdk/io/solr/SolrIOTest.java | 5 +- sdks/java/javadoc/pom.xml | 5 + 13 files changed, 237 insertions(+), 162 deletions(-) create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java diff --git a/pom.xml b/pom.xml index e0ec1362a678..b0a7376b8238 100644 --- a/pom.xml +++ b/pom.xml @@ -522,7 +522,13 @@ ${project.version} - + + org.apache.beam + beam-sdks-java-io-solr + ${project.version} + + + org.apache.beam beam-sdks-java-io-hadoop-input-format ${project.version} diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java index d317be9427bf..6470898233a9 100644 --- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java +++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java @@ -74,8 +74,8 @@ public interface IOTestPipelineOptions extends TestPipelineOptions { /* Solr */ @Description("Address of Zookeeper server for Solr") @Default.String("zookeeper-server") - String getZookeeperSolrServer(); - void setZookeeperSolrServer(String value); + String getSolrZookeeperServer(); + void getSolrZookeeperServer(String value); /* Cassandra */ @Description("Host for Cassandra server (host name/ip address)") diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index 143a8b44b072..1f9f2f6310a0 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -28,7 +28,7 @@ beam-sdks-java-io-solr Apache Beam :: SDKs :: Java :: IO :: Solr - IO to read and write on Solr. + IO to read and write from/to Solr. @@ -57,11 +57,6 @@ joda-time - - org.slf4j - slf4j-api - - com.google.auto.value @@ -70,7 +65,6 @@ - org.hamcrest hamcrest-core @@ -109,6 +103,12 @@ test + + org.slf4j + slf4j-api + test + + com.carrotsearch.randomizedtesting randomizedtesting-runner @@ -119,7 +119,7 @@ org.slf4j slf4j-log4j12 - 1.7.10 + ${slf4j.version} test diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java index b78bfdcd2aa0..228a123db9d9 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.solr; import java.io.IOException; @@ -14,25 +31,25 @@ /** * Client for interact with SolrCloud. */ -public class AuthorizedCloudSolrClient extends AuthorizedSolrClient { +class AuthorizedCloudSolrClient extends AuthorizedSolrClient { - public AuthorizedCloudSolrClient(CloudSolrClient solrClient, + AuthorizedCloudSolrClient(CloudSolrClient solrClient, SolrIO.ConnectionConfiguration configuration) { super(solrClient, configuration); } - public DocCollection getDocCollection(String collection){ + DocCollection getDocCollection(String collection){ solrClient.connect(); return solrClient.getZkStateReader().getClusterState().getCollection(collection); } - public QueryResponse query(String collection, SolrParams solrParams) + QueryResponse query(String collection, SolrParams solrParams) throws IOException, SolrServerException { QueryRequest query = new QueryRequest(solrParams); return process(collection, query); } - public T process(String collection, SolrRequest request) + T process(String collection, SolrRequest request) throws IOException, SolrServerException { request.setBasicAuthCredentials(username, password); return request.process(solrClient, collection); diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java index 81e4e7d1eb3a..75e0be15c2bb 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.beam.sdk.io.solr; import static com.google.common.base.Preconditions.checkArgument; @@ -23,7 +40,7 @@ class AuthorizedSolrClient implements Closeable { protected String username; protected String password; - public AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) { + AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) { checkArgument( solrClient != null, "AuthorizedSolrClient(solrClient, configuration) " @@ -37,13 +54,13 @@ public AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) this.password = configuration.getPassword(); } - public QueryResponse query(SolrParams solrParams) + QueryResponse query(SolrParams solrParams) throws IOException, SolrServerException { QueryRequest query = new QueryRequest(solrParams); return process(query); } - public T process(SolrRequest request) + T process(SolrRequest request) throws IOException, SolrServerException { request.setBasicAuthCredentials(username, password); return request.process(solrClient); diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java new file mode 100644 index 000000000000..076c841a5349 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solr; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.util.VarInt; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.util.JavaBinCodec; + +/** + * A {@link Coder} that encodes {@link SolrDocument SolrDocument}. + */ +class SolrDocumentCoder extends AtomicCoder { + + private static final SolrDocumentCoder INSTANCE = new SolrDocumentCoder(); + + public static SolrDocumentCoder of() { + return INSTANCE; + } + + @Override public void encode(SolrDocument value, OutputStream outStream) + throws CoderException, IOException { + if (value == null) { + throw new CoderException("cannot encode a null SolrDocument"); + } + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + JavaBinCodec codec = new JavaBinCodec(); + codec.marshal(value, baos); + + byte[] bytes = baos.toByteArray(); + VarInt.encode(bytes.length, outStream); + outStream.write(bytes); + } + + @Override public SolrDocument decode(InputStream inStream) throws CoderException, IOException { + DataInputStream in = new DataInputStream(inStream); + + int len = VarInt.decodeInt(in); + if (len < 0) { + throw new CoderException("Invalid encoded SolrDocument length: " + len); + } + byte[] bytes = new byte[len]; + in.readFully(bytes); + + JavaBinCodec codec = new JavaBinCodec(); + return (SolrDocument) codec.unmarshal(new ByteArrayInputStream(bytes)); + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 98793942bb06..504cc60cf401 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -24,17 +24,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.io.Serializable; import java.net.MalformedURLException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -43,14 +37,12 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -73,7 +65,6 @@ import org.apache.solr.common.params.CoreAdminParams; import org.apache.solr.common.params.CursorMarkParams; import org.apache.solr.common.params.ModifiableSolrParams; -import org.apache.solr.common.util.JavaBinCodec; import org.apache.solr.common.util.NamedList; /** @@ -120,7 +111,7 @@ *

Optionally, you can provide {@code withBatchSize()} * to specify the size of the write batch in number of documents. */ -@Experimental +@Experimental(Experimental.Kind.SOURCE_SINK) public class SolrIO { /** @@ -160,30 +151,21 @@ private SolrIO() { } /** A POJO describing a connection configuration to Solr. */ - @AutoValue public abstract static class ConnectionConfiguration implements Serializable { + @AutoValue + public abstract static class ConnectionConfiguration implements Serializable { abstract String getZkHost(); - - @Nullable - abstract String getUsername(); - - @Nullable - abstract String getPassword(); - abstract String getCollection(); - + @Nullable abstract String getUsername(); + @Nullable abstract String getPassword(); abstract Builder builder(); - @AutoValue.Builder abstract static class Builder { - + @AutoValue.Builder + abstract static class Builder { abstract Builder setZkHost(String zkHost); - + abstract Builder setCollection(String collection); abstract Builder setUsername(String username); - abstract Builder setPassword(String password); - - abstract Builder setCollection(String collection); - abstract ConnectionConfiguration build(); } @@ -203,12 +185,10 @@ public static ConnectionConfiguration create(String zkHost, String collection) checkArgument(collection != null, "ConnectionConfiguration.create(zkHost, collection) " + "called with null collectioin"); - ConnectionConfiguration connectionConfig = new AutoValue_SolrIO_ConnectionConfiguration - .Builder() + return new AutoValue_SolrIO_ConnectionConfiguration.Builder() .setZkHost(zkHost) .setCollection(collection) .build(); - return connectionConfig; } /** @@ -266,27 +246,20 @@ AuthorizedSolrClient createClient(String shardUrl) { } /** A {@link PTransform} reading data from Solr. */ - @AutoValue public abstract static class Read - extends PTransform> { - + @AutoValue + public abstract static class Read extends PTransform> { private static final long MAX_BATCH_SIZE = 10000L; abstract ConnectionConfiguration getConnectionConfiguration(); - abstract String getQuery(); - abstract long getBatchSize(); - abstract Builder builder(); - @AutoValue.Builder abstract static class Builder { - + @AutoValue.Builder + abstract static class Builder { abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); - abstract Builder setQuery(String query); - abstract Builder setBatchSize(long batchSize); - abstract Read build(); } @@ -325,18 +298,21 @@ public Read withBatchSize(long batchSize) { return builder().setBatchSize(batchSize).build(); } - @Override public PCollection expand(PBegin input) { + @Override + public PCollection expand(PBegin input) { return input.apply(org.apache.beam.sdk.io.Read .from(new BoundedSolrSource(this, null))); } - @Override public void validate(PipelineOptions options) { + @Override + public void validate(PipelineOptions options) { checkState(getConnectionConfiguration() != null, "SolrIO.read() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); } - @Override public void populateDisplayData(DisplayData.Builder builder) { + @Override + public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder.addIfNotNull(DisplayData.item("query", getQuery())); getConnectionConfiguration().populateDisplayData(builder); @@ -357,7 +333,8 @@ static ReplicaInfo create(Replica replica) { } /** A {@link BoundedSource} reading from Solr. */ - @VisibleForTesting static class BoundedSolrSource extends BoundedSource { + @VisibleForTesting + static class BoundedSolrSource extends BoundedSource { private final SolrIO.Read spec; // replica is the info of the shard where the source will read the documents @@ -368,7 +345,8 @@ static ReplicaInfo create(Replica replica) { this.replica = replica == null ? null : ReplicaInfo.create(replica); } - @Override public List> split(long desiredBundleSizeBytes, + @Override + public List> split(long desiredBundleSizeBytes, PipelineOptions options) throws Exception { List sources = new ArrayList<>(); int numShard; @@ -384,7 +362,8 @@ static ReplicaInfo create(Replica replica) { return sources; } - @Override public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { if (replica != null) { return getEstimatedSizeOfShard(replica); } else { @@ -423,70 +402,28 @@ private long getEstimatedSizeOfCollection() throws IOException { return sizeInBytes; } - @Override public void populateDisplayData(DisplayData.Builder builder) { + @Override + public void populateDisplayData(DisplayData.Builder builder) { spec.populateDisplayData(builder); if (replica != null) { builder.addIfNotNull(DisplayData.item("shardUrl", replica.coreUrl())); } } - @Override public BoundedReader createReader(PipelineOptions options) + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { return new BoundedSolrReader(this); } - @Override public void validate() { + @Override + public void validate() { spec.validate(null); } - @Override public Coder getDefaultOutputCoder() { - return SolrCoder.of(); - } - } - - /** - * A {@link Coder} that encodes {@link SolrDocument SolrDocument}. - */ - public static class SolrCoder extends Coder { - - private static final SolrCoder INSTANCE = new SolrCoder(); - - public static SolrCoder of() { - return INSTANCE; - } - - @Override public void encode(SolrDocument value, OutputStream outStream) - throws CoderException, IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - - JavaBinCodec codec = new JavaBinCodec(); - codec.marshal(value, baos); - - byte[] bytes = baos.toByteArray(); - VarInt.encode(bytes.length, outStream); - outStream.write(bytes); - } - - @Override public SolrDocument decode(InputStream inStream) throws CoderException, IOException { - DataInputStream in = new DataInputStream(inStream); - - int len = VarInt.decodeInt(in); - if (len < 0) { - throw new CoderException("Invalid encoded SolrDocument length: " + len); - } - byte[] bytes = new byte[len]; - in.readFully(bytes); - - JavaBinCodec codec = new JavaBinCodec(); - return (SolrDocument) codec.unmarshal(new ByteArrayInputStream(bytes)); - } - - @Override public List> getCoderArguments() { - return Collections.emptyList(); - } - - @Override public void verifyDeterministic() throws NonDeterministicException { - + @Override + public Coder getDefaultOutputCoder() { + return SolrDocumentCoder.of(); } } @@ -505,7 +442,8 @@ private BoundedSolrReader(BoundedSolrSource source) { this.cursorMark = CursorMarkParams.CURSOR_MARK_START; } - @Override public boolean start() throws IOException { + @Override + public boolean start() throws IOException { if (source.replica != null) { solrClient = source.spec.getConnectionConfiguration() .createClient(source.replica.coreUrl()); @@ -545,7 +483,8 @@ private void updateCursorMark(QueryResponse response) { cursorMark = response.getNextCursorMark(); } - @Override public boolean advance() throws IOException { + @Override + public boolean advance() throws IOException { if (batchIterator.hasNext()) { current = batchIterator.next(); return true; @@ -573,38 +512,37 @@ private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { return true; } - @Override public SolrDocument getCurrent() throws NoSuchElementException { + @Override + public SolrDocument getCurrent() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } return current; } - @Override public void close() throws IOException { + @Override + public void close() throws IOException { solrClient.close(); } - @Override public BoundedSource getCurrentSource() { + @Override + public BoundedSource getCurrentSource() { return source; } } /** A {@link PTransform} writing data to Solr. */ - @AutoValue public abstract static class Write - extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { abstract ConnectionConfiguration getConnectionConfiguration(); - abstract long getMaxBatchSize(); - abstract Builder builder(); - @AutoValue.Builder abstract static class Builder { - + @AutoValue.Builder + abstract static class Builder { abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); - abstract Builder setMaxBatchSize(long maxBatchSize); - abstract Write build(); } @@ -622,18 +560,21 @@ public Write withMaxBatchSize(long batchSize) { return builder().setMaxBatchSize(batchSize).build(); } - @Override public void validate(PipelineOptions options) { + @Override + public void validate(PipelineOptions options) { checkState(getConnectionConfiguration() != null, "SolrIO.write() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); } - @Override public PDone expand(PCollection input) { + @Override + public PDone expand(PCollection input) { input.apply(ParDo.of(new WriteFn(this))); return PDone.in(input.getPipeline()); } - @VisibleForTesting static class WriteFn extends DoFn { + @VisibleForTesting + static class WriteFn extends DoFn { private final Write spec; @@ -644,15 +585,18 @@ public Write withMaxBatchSize(long batchSize) { this.spec = spec; } - @Setup public void createClient() throws Exception { + @Setup + public void createClient() throws Exception { solrClient = spec.getConnectionConfiguration().createClient(); } - @StartBundle public void startBundle(StartBundleContext context) throws Exception { + @StartBundle + public void startBundle(StartBundleContext context) throws Exception { batch = new ArrayList<>(); } - @ProcessElement public void processElement(ProcessContext context) throws Exception { + @ProcessElement + public void processElement(ProcessContext context) throws Exception { SolrInputDocument document = context.element(); batch.add(document); if (batch.size() >= spec.getMaxBatchSize()) { diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java index 3d5770e9e14e..83867ed0f443 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java @@ -16,5 +16,5 @@ * limitations under the License. */ -/** Transforms for reading and writing from Solr. */ +/** Transforms for reading and writing from/to Solr. */ package org.apache.beam.sdk.io.solr; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java index ebcc6c608d1d..1dceaa79151d 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java @@ -18,13 +18,15 @@ package org.apache.beam.sdk.io.solr; import com.carrotsearch.randomizedtesting.ThreadFilter; +import com.google.common.annotations.VisibleForTesting; /** * Filter out Beam leak threads. */ +@VisibleForTesting public class BeamThreadsFilter implements ThreadFilter { @Override public boolean reject(Thread t) { return t.getName().startsWith("direct-metrics-counter"); } -} +} \ No newline at end of file diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java index 5a7bc0be4faf..9bd8f9d0a52e 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java @@ -15,33 +15,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.sdk.io.solr; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; -import com.google.common.io.CountingInputStream; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.List; + import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.UnownedInputStream; -import org.apache.beam.sdk.util.UnownedOutputStream; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.solr.common.SolrDocument; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** - * Test case for {@link SolrIO.SolrCoder}. + * Test case for {@link SolrDocumentCoder}. */ @RunWith(JUnit4.class) public class SolrIOCoderTest { - - private static final Coder TEST_CODER = SolrIO.SolrCoder.of(); - + private static final Coder TEST_CODER = SolrDocumentCoder.of(); private static final List TEST_VALUES = new ArrayList<>(); static { @@ -58,17 +57,28 @@ public class SolrIOCoderTest { TEST_VALUES.add(doc); } - @Test public void testDecodeEncodeEqual() throws Exception { + @Test + public void testDecodeEncodeEqual() throws Exception { for (SolrDocument value : TEST_VALUES) { - ByteArrayOutputStream os = new ByteArrayOutputStream(); - TEST_CODER.encode(value, new UnownedOutputStream(os)); - byte[] bytes = os.toByteArray(); - CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(bytes)); - SolrDocument decoded = TEST_CODER.decode(new UnownedInputStream(cis)); - assertThat("consumed bytes equal to encoded bytes", cis.getCount(), - equalTo((long) bytes.length)); - assertThat(decoded.entrySet(), equalTo(value.entrySet())); - assertThat(decoded.getChildDocuments(), equalTo(value.getChildDocuments())); + CoderProperties.coderDecodeEncodeContentsInSameOrder(TEST_CODER, value); + CoderProperties.structuralValueDecodeEncodeEqual(TEST_CODER, value); } } + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void encodeNullThrowsCoderException() throws Exception { + thrown.expect(CoderException.class); + thrown.expectMessage("cannot encode a null SolrDocument"); + + CoderUtils.encodeToBase64(TEST_CODER, null); + } + + @Test + public void testEncodedTypeDescriptor() throws Exception { + assertThat(TEST_CODER.getEncodedTypeDescriptor(), + equalTo(TypeDescriptor.of(SolrDocument.class))); + } } diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index 64dc7a121c21..858fda198f89 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -56,7 +56,7 @@ * *

  *  mvn -e -Pio-it verify -pl sdks/java/io/solr -DintegrationTestPipelineOptions='[
- *  "--zookeeperSolrServer=127.0.0.1:9983"]'
+ *  "--solrZookeeperServer=127.0.0.1:9983"]'
  * 
*/ public class SolrIOIT { @@ -77,7 +77,7 @@ public static void beforeClass() throws Exception { PipelineOptionsFactory.register(IOTestPipelineOptions.class); options = TestPipeline.testingPipelineOptions().as(IOTestPipelineOptions.class); readConnectionConfiguration = SolrIO.ConnectionConfiguration - .create(options.getZookeeperSolrServer(), READ_COLLECTION); + .create(options.getSolrZookeeperServer(), READ_COLLECTION); solrClient = readConnectionConfiguration.createClient(); CollectionAdminResponse listResponse = solrClient.process(new CollectionAdminRequest.List()); List collections = (List) listResponse.getResponse().get("collections"); @@ -140,7 +140,7 @@ public void testReadVolume() throws Exception { @Test public void testWriteVolume() throws Exception { SolrIO.ConnectionConfiguration writeConnectionConfiguration = - SolrIO.ConnectionConfiguration.create(options.getZookeeperSolrServer(), WRITE_COLLECTION); + SolrIO.ConnectionConfiguration.create(options.getSolrZookeeperServer(), WRITE_COLLECTION); List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 5464e146c230..adb644f12a5d 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -57,13 +57,13 @@ * A test of {@link SolrIO} on an independent Solr instance. */ @ThreadLeakFilters(defaultFilters = true, filters = { - BeamThreadsFilter.class + BeamThreadsFilter.class }) public class SolrIOTest extends SolrCloudTestCase{ private static final Logger LOG = LoggerFactory.getLogger(SolrIOTest.class); private static final String SOLR_COLLECTION = "beam"; - public static final int NUM_SHARDS = 3; + private static final int NUM_SHARDS = 3; private static final long NUM_DOCS = 400L; private static final int NUM_SCIENTISTS = 10; private static final long BATCH_SIZE = 200L; @@ -89,6 +89,7 @@ public static void beforeClass() throws Exception { ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader(); zkStateReader.getZkClient() .setData("/security.json", securityJson.getBytes(Charset.defaultCharset()), true); + Thread.sleep(1000); TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS); while (!timeOut.hasTimedOut()) { if (zkStateReader.getClusterState().getLiveNodes().size() == 3) { diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index 51109fbe1c94..a4769fc95ecc 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -167,6 +167,11 @@ beam-sdks-java-io-mqtt
+ + org.apache.beam + beam-sdks-java-io-solr + + com.google.auto.service From 1a9c78510ef0088881b2411450b3d110485d96d0 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 28 Jul 2017 22:47:49 +0700 Subject: [PATCH 07/14] Remove needed a ConnectionConfiguration object for SolrIO.read() and Solr.write() --- .../org/apache/beam/sdk/io/solr/SolrIO.java | 61 +++++++++---------- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 8 +-- .../apache/beam/sdk/io/solr/SolrIOTest.java | 15 +++-- 3 files changed, 43 insertions(+), 41 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 504cc60cf401..734658e2fba9 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -114,37 +114,12 @@ @Experimental(Experimental.Kind.SOURCE_SINK) public class SolrIO { - /** - * Create new SolrIO.Read based on provided Solr connection configuration object. - * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Read} with connection configuration set - */ - public static Read read(ConnectionConfiguration connectionConfiguration) { - checkArgument( - connectionConfiguration != null, - "SolrIO.read(connectionConfiguration) " - + "called with null connectionConfiguration"); - return new AutoValue_SolrIO_Read.Builder() - .setConnectionConfiguration(connectionConfiguration) - .setBatchSize(100L) - .setQuery("*:*") - .build(); + public static Read read() { + return new AutoValue_SolrIO_Read.Builder().setBatchSize(100L).setQuery("*:*").build(); } - /** - * Create new SolrIO.Write based on provided Solr connection configuration object. - * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Write} with connection configuration set - */ - public static Write write(ConnectionConfiguration connectionConfiguration) { - checkArgument( - connectionConfiguration != null, - "SolrIO.write(connectionConfiguration) " - + "called with null connectionConfiguration"); - return new AutoValue_SolrIO_Write.Builder() - .setMaxBatchSize(1000L) - .setConnectionConfiguration(connectionConfiguration) - .build(); + public static Write write() { + return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000L).build(); } private SolrIO() { @@ -250,7 +225,7 @@ AuthorizedSolrClient createClient(String shardUrl) { public abstract static class Read extends PTransform> { private static final long MAX_BATCH_SIZE = 10000L; - abstract ConnectionConfiguration getConnectionConfiguration(); + @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); abstract String getQuery(); abstract long getBatchSize(); abstract Builder builder(); @@ -263,6 +238,18 @@ abstract static class Builder { abstract Read build(); } + /** + * Provide the Solr connection configuration object. + * + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Read} with connection configuration set + */ + public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "SolrIO.read()" + + ".withConnectionConfiguration(configuration) called with null configuration"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + /** * Provide a query used while reading from Solr. * @@ -535,7 +522,7 @@ public BoundedSource getCurrentSource() { @AutoValue public abstract static class Write extends PTransform, PDone> { - abstract ConnectionConfiguration getConnectionConfiguration(); + @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); abstract long getMaxBatchSize(); abstract Builder builder(); @@ -546,6 +533,18 @@ abstract static class Builder { abstract Write build(); } + /** + * Provide the Solr connection configuration object. + * + * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object + * @return the {@link Write} with connection configuration set + */ + public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "SolrIO.write()" + + ".withConnectionConfiguration(configuration) called with null configuration"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + /** * Provide a maximum size in number of documents for the batch. Depending on the * execution engine, size of bundles may vary, this sets the maximum size. Change this if you diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index 858fda198f89..bd720268f4a3 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -107,7 +107,7 @@ public static void afterClass() throws Exception { @Test public void testSplitsVolume() throws Exception { SolrIO.Read read = - SolrIO.read(readConnectionConfiguration); + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored now @@ -131,7 +131,7 @@ public void testSplitsVolume() throws Exception { public void testReadVolume() throws Exception { PCollection output = pipeline.apply( - SolrIO.read(readConnectionConfiguration)); + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); pipeline.run(); @@ -144,7 +144,7 @@ public void testWriteVolume() throws Exception { List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) - .apply(SolrIO.write(writeConnectionConfiguration)); + .apply(SolrIO.write().withConnectionConfiguration(writeConnectionConfiguration)); pipeline.run(); long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs( @@ -155,7 +155,7 @@ public void testWriteVolume() throws Exception { @Test public void testEstimatedSizesVolume() throws Exception { SolrIO.Read read = - SolrIO.read(readConnectionConfiguration); + SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index adb644f12a5d..7cd300f75c9c 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -135,7 +135,7 @@ public void testSizes() throws Exception { SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read(connectionConfiguration); + SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size // (due to internal Lucene implementation) @@ -157,7 +157,8 @@ public void testRead() throws Exception { PCollection output = pipeline.apply( - SolrIO.read(connectionConfiguration) + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) .withBatchSize(100L)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); @@ -170,7 +171,8 @@ public void testReadWithQuery() throws Exception { PCollection output = pipeline.apply( - SolrIO.read(connectionConfiguration) + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) .withQuery("scientist:Einstein")); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); @@ -182,7 +184,7 @@ public void testWrite() throws Exception { List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) - .apply(SolrIO.write(connectionConfiguration)); + .apply(SolrIO.write().withConnectionConfiguration(connectionConfiguration)); pipeline.run(); long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); @@ -195,7 +197,8 @@ public void testWrite() throws Exception { @Test public void testWriteWithMaxBatchSize() throws Exception { SolrIO.Write write = - SolrIO.write(connectionConfiguration) + SolrIO.write() + .withConnectionConfiguration(connectionConfiguration) .withMaxBatchSize(BATCH_SIZE); // write bundles size is the runner decision, we cannot force a bundle size, // so we test the Writer as a DoFn outside of a runner. @@ -238,7 +241,7 @@ public void testSplit() throws Exception { SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read(connectionConfiguration); + SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored for now int desiredBundleSizeBytes = 0; From 9361debbe19e3f76ee52c64086c8a8e5befc3b22 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 28 Jul 2017 23:04:19 +0700 Subject: [PATCH 08/14] SolrIO: Fix mvn clean verify errors --- sdks/java/io/solr/pom.xml | 14 ++++++++++++++ .../apache/beam/sdk/io/solr/BeamThreadsFilter.java | 2 +- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index 1f9f2f6310a0..f08198a607ff 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -64,6 +64,13 @@ provided + + org.apache.httpcomponents + httpclient + 4.4.1 + provided + + org.hamcrest @@ -103,6 +110,13 @@ test + + org.apache.solr + solr-core + 5.5.4 + test + + org.slf4j slf4j-api diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java index 1dceaa79151d..e6e1bcd657e0 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java @@ -29,4 +29,4 @@ public class BeamThreadsFilter implements ThreadFilter { @Override public boolean reject(Thread t) { return t.getName().startsWith("direct-metrics-counter"); } -} \ No newline at end of file +} From a5a4862d61bad618059876f0f0f3005f226f9749 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Sun, 30 Jul 2017 08:37:59 +0700 Subject: [PATCH 09/14] SolrIO: Update based on Eugene's reviews --- .../sdk/io/common/IOTestPipelineOptions.java | 2 +- sdks/java/io/solr/pom.xml | 5 + .../io/solr/AuthorizedCloudSolrClient.java | 57 --------- .../sdk/io/solr/AuthorizedSolrClient.java | 19 ++- .../beam/sdk/io/solr/SolrDocumentCoder.java | 14 +-- .../org/apache/beam/sdk/io/solr/SolrIO.java | 108 +++++++++++++----- .../beam/sdk/io/solr/BeamThreadsFilter.java | 32 ------ ...erTest.java => SolrDocumentCoderTest.java} | 2 +- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 24 ++-- .../apache/beam/sdk/io/solr/SolrIOTest.java | 75 ++++++------ .../beam/sdk/io/solr/SolrIOTestUtils.java | 28 ++--- 11 files changed, 173 insertions(+), 193 deletions(-) delete mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java delete mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java rename sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/{SolrIOCoderTest.java => SolrDocumentCoderTest.java} (98%) diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java index 6470898233a9..256c94d8019f 100644 --- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java +++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java @@ -75,7 +75,7 @@ public interface IOTestPipelineOptions extends TestPipelineOptions { @Description("Address of Zookeeper server for Solr") @Default.String("zookeeper-server") String getSolrZookeeperServer(); - void getSolrZookeeperServer(String value); + void setSolrZookeeperServer(String value); /* Cassandra */ @Description("Host for Cassandra server (host name/ip address)") diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index f08198a607ff..fa112add90b7 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -57,6 +57,11 @@ joda-time + + org.apache.commons + commons-compress + + com.google.auto.value diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java deleted file mode 100644 index 228a123db9d9..000000000000 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedCloudSolrClient.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solr; - -import java.io.IOException; - -import org.apache.solr.client.solrj.SolrRequest; -import org.apache.solr.client.solrj.SolrResponse; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.impl.CloudSolrClient; -import org.apache.solr.client.solrj.request.QueryRequest; -import org.apache.solr.client.solrj.response.QueryResponse; -import org.apache.solr.common.cloud.DocCollection; -import org.apache.solr.common.params.SolrParams; - -/** - * Client for interact with SolrCloud. - */ -class AuthorizedCloudSolrClient extends AuthorizedSolrClient { - - AuthorizedCloudSolrClient(CloudSolrClient solrClient, - SolrIO.ConnectionConfiguration configuration) { - super(solrClient, configuration); - } - - DocCollection getDocCollection(String collection){ - solrClient.connect(); - return solrClient.getZkStateReader().getClusterState().getCollection(collection); - } - - QueryResponse query(String collection, SolrParams solrParams) - throws IOException, SolrServerException { - QueryRequest query = new QueryRequest(solrParams); - return process(collection, query); - } - - T process(String collection, SolrRequest request) - throws IOException, SolrServerException { - request.setBasicAuthCredentials(username, password); - return request.process(solrClient, collection); - } -} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java index 75e0be15c2bb..74813ffe2cf5 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java @@ -27,18 +27,21 @@ import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.params.SolrParams; /** - * Client for interact with a replica in Solr. + * Client for interact with Solr. + * The target replica/collection is pre-selected * @param type of SolrClient */ class AuthorizedSolrClient implements Closeable { - protected T solrClient; - protected String username; - protected String password; + private final T solrClient; + private final String username; + private final String password; AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) { checkArgument( @@ -60,12 +63,18 @@ QueryResponse query(SolrParams solrParams) return process(query); } - T process(SolrRequest request) + T2 process(SolrRequest request) throws IOException, SolrServerException { request.setBasicAuthCredentials(username, password); return request.process(solrClient); } + static ClusterState getClusterState( + AuthorizedSolrClient authorizedSolrClient) { + authorizedSolrClient.solrClient.connect(); + return authorizedSolrClient.solrClient.getZkStateReader().getClusterState(); + } + @Override public void close() throws IOException { solrClient.close(); } diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java index 076c841a5349..9838fc39b281 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrDocumentCoder.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.solr; -import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.IOException; @@ -28,6 +27,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.VarInt; +import org.apache.commons.compress.utils.BoundedInputStream; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.util.JavaBinCodec; @@ -42,8 +42,9 @@ public static SolrDocumentCoder of() { return INSTANCE; } - @Override public void encode(SolrDocument value, OutputStream outStream) - throws CoderException, IOException { + @Override + public void encode(SolrDocument value, OutputStream outStream) + throws IOException { if (value == null) { throw new CoderException("cannot encode a null SolrDocument"); } @@ -57,17 +58,16 @@ public static SolrDocumentCoder of() { outStream.write(bytes); } - @Override public SolrDocument decode(InputStream inStream) throws CoderException, IOException { + @Override + public SolrDocument decode(InputStream inStream) throws IOException { DataInputStream in = new DataInputStream(inStream); int len = VarInt.decodeInt(in); if (len < 0) { throw new CoderException("Invalid encoded SolrDocument length: " + len); } - byte[] bytes = new byte[len]; - in.readFully(bytes); JavaBinCodec codec = new JavaBinCodec(); - return (SolrDocument) codec.unmarshal(new ByteArrayInputStream(bytes)); + return (SolrDocument) codec.unmarshal(new BoundedInputStream(in, len)); } } diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 734658e2fba9..50a5e68be6f3 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -23,15 +23,23 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.io.Serializable; import java.net.MalformedURLException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -58,6 +66,7 @@ import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrInputDocument; +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; @@ -150,10 +159,8 @@ abstract static class Builder { * @param zkHost host of zookeeper * @param collection the collection toward which the requests will be issued * @return the connection configuration object - * @throws IOException when it fails to connect to Solr */ - public static ConnectionConfiguration create(String zkHost, String collection) - throws IOException { + public static ConnectionConfiguration create(String zkHost, String collection) { checkArgument(zkHost != null, "ConnectionConfiguration.create(zkHost, collection) " + "called with null address"); @@ -168,10 +175,6 @@ public static ConnectionConfiguration create(String zkHost, String collection) /** * If Solr basic authentication is enabled, provide the username and password. - * - * @param username the username used to authenticate to Solr - * @param password the password used to authenticate to Solr - * @return the {@link ConnectionConfiguration} object with basic credentials set */ public ConnectionConfiguration withBasicCredentials(String username, String password) { checkArgument( @@ -208,13 +211,13 @@ private HttpClient createHttpClient() { return HttpClientUtil.createClient(params); } - AuthorizedCloudSolrClient createClient() throws MalformedURLException { + AuthorizedSolrClient createClient() throws MalformedURLException { CloudSolrClient solrClient = new CloudSolrClient(getZkHost(), createHttpClient()); solrClient.setDefaultCollection(getCollection()); - return new AuthorizedCloudSolrClient(solrClient, this); + return new AuthorizedSolrClient<>(solrClient, this); } - AuthorizedSolrClient createClient(String shardUrl) { + AuthorizedSolrClient createClient(String shardUrl) { HttpSolrClient solrClient = new HttpSolrClient(shardUrl, createHttpClient()); return new AuthorizedSolrClient<>(solrClient, this); } @@ -242,7 +245,6 @@ abstract static class Builder { * Provide the Solr connection configuration object. * * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Read} with connection configuration set */ public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { checkArgument(connectionConfiguration != null, "SolrIO.read()" @@ -257,7 +259,6 @@ public Read withConnectionConfiguration(ConnectionConfiguration connectionConfig * href="https://cwiki.apache.org/confluence/display/solr/The+Standard+Query+Parser"> * Solr Query * - * @return the {@link Read} object with query set */ public Read withQuery(String query) { checkArgument(!Strings.isNullOrEmpty(query), @@ -273,7 +274,6 @@ public Read withQuery(String query) { * batchSize * * @param batchSize number of documents read in each scroll read - * @return the {@link Read} with batch size set */ public Read withBatchSize(long batchSize) { checkArgument(batchSize > 0, "SolrIO.read().withBatchSize(batchSize) " @@ -335,17 +335,31 @@ static class BoundedSolrSource extends BoundedSource { @Override public List> split(long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + ConnectionConfiguration connectionConfig = spec.getConnectionConfiguration(); List sources = new ArrayList<>(); - int numShard; - try (AuthorizedCloudSolrClient client = spec.getConnectionConfiguration().createClient()) { - String collection = spec.getConnectionConfiguration().getCollection(); - DocCollection docCollection = client.getDocCollection(collection); - numShard = docCollection.getSlices().size(); + try (AuthorizedSolrClient client = connectionConfig.createClient()) { + String collection = connectionConfig.getCollection(); + final ClusterState clusterState = AuthorizedSolrClient.getClusterState(client); + DocCollection docCollection = clusterState.getCollection(collection); for (Slice slice : docCollection.getSlices()) { - sources.add(new BoundedSolrSource(spec, slice.getLeader())); + ArrayList replicas = new ArrayList<>(slice.getReplicas()); + Collections.shuffle(replicas); + // Load balancing by randomly picking an active replica + Replica randomActiveReplica = null; + for (Replica replica : replicas) { + // We need to check both state of the replica and live nodes + // to make sure that the replica is alive + if (replica.getState() == Replica.State.ACTIVE + && clusterState.getLiveNodes().contains(replica.getNodeName())) { + randomActiveReplica = replica; + break; + } + } + checkState(randomActiveReplica != null, + "Can not found an active replica for slice %s", slice.getName()); + sources.add(new BoundedSolrSource(spec, randomActiveReplica)); } } - checkArgument(sources.size() == numShard, "Not enough leaders were found"); return sources; } @@ -379,14 +393,52 @@ private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOExce private long getEstimatedSizeOfCollection() throws IOException { long sizeInBytes = 0; ConnectionConfiguration config = spec.getConnectionConfiguration(); - try (AuthorizedCloudSolrClient solrClient = config.createClient()) { - DocCollection docCollection = solrClient.getDocCollection(config.getCollection()); - for (Slice slice : docCollection.getSlices()) { - Replica replica = slice.getLeader(); - sizeInBytes += getEstimatedSizeOfShard(ReplicaInfo.create(replica)); + try (AuthorizedSolrClient solrClient = config.createClient()) { + DocCollection docCollection = AuthorizedSolrClient.getClusterState(solrClient) + .getCollection(config.getCollection()); + if (docCollection.getSlices().isEmpty()) { + return 0; + } + + ArrayList slices = new ArrayList<>(docCollection.getSlices()); + Collections.shuffle(slices); + ExecutorService executor = Executors.newCachedThreadPool( + new ThreadFactoryBuilder() + .setThreadFactory(MoreExecutors.platformThreadFactory()) + .setDaemon(true) + .setNameFormat("solrio-size-of-collection-estimation") + .build()); + try { + ArrayList> futures = new ArrayList<>(); + for (int i = 0; i < 100 && i < slices.size(); i++) { + Slice slice = slices.get(i); + final Replica replica = slice.getLeader(); + Future future = executor.submit(new Callable() { + @Override public Long call() throws Exception { + return getEstimatedSizeOfShard(ReplicaInfo.create(replica)); + } + }); + futures.add(future); + } + for (Future future : futures) { + try { + sizeInBytes += future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException e) { + throw new IOException("Can not estimate size of shard", e.getCause()); + } + } + } finally { + executor.shutdownNow(); + } + + if (slices.size() <= 100) { + return sizeInBytes; } + return (sizeInBytes / 100) * slices.size(); } - return sizeInBytes; } @Override @@ -488,7 +540,7 @@ public boolean advance() throws IOException { } private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { - if (response.getResults().isEmpty() || done) { + if (done) { current = null; batchIterator = null; return false; @@ -537,7 +589,6 @@ abstract static class Builder { * Provide the Solr connection configuration object. * * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object - * @return the {@link Write} with connection configuration set */ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { checkArgument(connectionConfiguration != null, "SolrIO.write()" @@ -551,7 +602,6 @@ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfi * need to have smaller batch. * * @param batchSize maximum batch size in number of documents - * @return the {@link Write} with connection batch size set */ public Write withMaxBatchSize(long batchSize) { checkArgument(batchSize > 0, diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java deleted file mode 100644 index e6e1bcd657e0..000000000000 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/BeamThreadsFilter.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solr; - -import com.carrotsearch.randomizedtesting.ThreadFilter; -import com.google.common.annotations.VisibleForTesting; - -/** - * Filter out Beam leak threads. - */ -@VisibleForTesting -public class BeamThreadsFilter implements ThreadFilter { - - @Override public boolean reject(Thread t) { - return t.getName().startsWith("direct-metrics-counter"); - } -} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrDocumentCoderTest.java similarity index 98% rename from sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java rename to sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrDocumentCoderTest.java index 9bd8f9d0a52e..dc20c4621c3b 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOCoderTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrDocumentCoderTest.java @@ -39,7 +39,7 @@ * Test case for {@link SolrDocumentCoder}. */ @RunWith(JUnit4.class) -public class SolrIOCoderTest { +public class SolrDocumentCoderTest { private static final Coder TEST_CODER = SolrDocumentCoder.of(); private static final List TEST_VALUES = new ArrayList<>(); diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index bd720268f4a3..419058220a3a 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -67,7 +67,7 @@ public class SolrIOIT { public static final String WRITE_COLLECTION = "beam" + Instant.now().getMillis(); private static final Logger LOG = LoggerFactory.getLogger(SolrIOIT.class); - private static AuthorizedCloudSolrClient solrClient; + private static AuthorizedSolrClient solrClient; private static IOTestPipelineOptions options; private static SolrIO.ConnectionConfiguration readConnectionConfiguration; @Rule public TestPipeline pipeline = TestPipeline.create(); @@ -79,16 +79,17 @@ public static void beforeClass() throws Exception { readConnectionConfiguration = SolrIO.ConnectionConfiguration .create(options.getSolrZookeeperServer(), READ_COLLECTION); solrClient = readConnectionConfiguration.createClient(); - CollectionAdminResponse listResponse = solrClient.process(new CollectionAdminRequest.List()); + + CollectionAdminResponse listResponse = + (CollectionAdminResponse) solrClient.process(new CollectionAdminRequest.List()); List collections = (List) listResponse.getResponse().get("collections"); if (collections.contains(READ_COLLECTION)) { - QueryResponse response = solrClient.query( - READ_COLLECTION, new SolrQuery("*:*")); + QueryResponse response = solrClient.query(new SolrQuery("*:*")); if (response.getResults().getNumFound() != NUM_DOCS) { LOG.info("Collection {} is exist but the number of documents is not match, repopulate", READ_COLLECTION); - SolrIOTestUtils.clearCollection(READ_COLLECTION, solrClient); - SolrIOTestUtils.insertTestDocuments(READ_COLLECTION, NUM_DOCS, solrClient); + SolrIOTestUtils.clearCollection(solrClient); + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); } } else { LOG.info("Create and populate collection {}", @@ -124,6 +125,8 @@ public void testSplitsVolume() throws Exception { nonEmptySplits += 1; } } + // docs are hashed by id to shards, in this test, NUM_DOCS >> NUM_SHARDS + // therefore, can not exist an empty shard. assertEquals(expectedNumSplits, nonEmptySplits); } @@ -147,9 +150,10 @@ public void testWriteVolume() throws Exception { .apply(SolrIO.write().withConnectionConfiguration(writeConnectionConfiguration)); pipeline.run(); - long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs( - WRITE_COLLECTION, solrClient); - assertEquals(NUM_DOCS, currentNumDocs); + try (AuthorizedSolrClient solrClient = writeConnectionConfiguration.createClient()) { + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(solrClient); + assertEquals(NUM_DOCS, currentNumDocs); + } } @Test @@ -165,7 +169,7 @@ public void testEstimatedSizesVolume() throws Exception { assertThat( "Wrong estimated size bellow minimum", estimatedSize, - greaterThan(SolrIOTestUtils.AVERAGE_DOC_SIZE * NUM_DOCS)); + greaterThan(SolrIOTestUtils.MIN_DOC_SIZE * NUM_DOCS)); assertThat( "Wrong estimated size beyond maximum", estimatedSize, diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 7cd300f75c9c..434308b08215 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -20,12 +20,12 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; import static org.hamcrest.Matchers.greaterThan; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; +import com.google.common.io.BaseEncoding; import java.io.IOException; import java.nio.charset.Charset; import java.util.List; -import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -37,28 +37,30 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.PCollection; +import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.ZkStateReader; -import org.apache.solr.util.TimeOut; +import org.apache.solr.security.Sha256AuthenticationProvider; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * A test of {@link SolrIO} on an independent Solr instance. */ -@ThreadLeakFilters(defaultFilters = true, filters = { - BeamThreadsFilter.class -}) +@ThreadLeakScope(value = ThreadLeakScope.Scope.NONE) +@SolrTestCaseJ4.SuppressSSL public class SolrIOTest extends SolrCloudTestCase{ private static final Logger LOG = LoggerFactory.getLogger(SolrIOTest.class); @@ -68,7 +70,7 @@ public class SolrIOTest extends SolrCloudTestCase{ private static final int NUM_SCIENTISTS = 10; private static final long BATCH_SIZE = 200L; - private static AuthorizedCloudSolrClient solrClient; + private static AuthorizedSolrClient solrClient; private static SolrIO.ConnectionConfiguration connectionConfiguration; @Rule @@ -76,37 +78,30 @@ public class SolrIOTest extends SolrCloudTestCase{ @BeforeClass public static void beforeClass() throws Exception { + //setup credential for solr user + String password = "SolrRocks"; + byte[] salt = new byte[32]; + String base64Salt = BaseEncoding.base64().encode(salt); + String sha56 = Sha256AuthenticationProvider.sha256(password, base64Salt); + String credential = sha56 + " " + base64Salt; String securityJson = "{" + "'authentication':{" + " 'blockUnknown': true," + " 'class':'solr.BasicAuthPlugin'," - + " 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=" - + " Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}}" + + " 'credentials':{'solr':'" + credential + "'}}" + "}"; + configureCluster(3) .addConfig("conf", getFile("cloud-minimal/conf").toPath()) .configure(); ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader(); zkStateReader.getZkClient() .setData("/security.json", securityJson.getBytes(Charset.defaultCharset()), true); - Thread.sleep(1000); - TimeOut timeOut = new TimeOut(60, TimeUnit.SECONDS); - while (!timeOut.hasTimedOut()) { - if (zkStateReader.getClusterState().getLiveNodes().size() == 3) { - break; - } else { - Thread.sleep(100); - } - } - if (timeOut.hasTimedOut()) { - fail("Timeout waiting for nodes come back"); - } String zkAddress = cluster.getZkServer().getZkAddress(); connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION) - .withBasicCredentials("solr", "SolrRocks"); + .withBasicCredentials("solr", password); solrClient = connectionConfiguration.createClient(); SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, 1, solrClient); - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); } @AfterClass @@ -116,23 +111,27 @@ public static void afterClass() throws Exception { @Before public void before() throws Exception { - SolrIOTestUtils.clearCollection(SOLR_COLLECTION, solrClient); + SolrIOTestUtils.clearCollection(solrClient); } - @Test(expected = SolrException.class) + @Rule + public ExpectedException thrown = ExpectedException.none(); + public void testBadCredentials() throws IOException { + thrown.expect(SolrException.class); + String zkAddress = cluster.getZkServer().getZkAddress(); SolrIO.ConnectionConfiguration connectionConfiguration = SolrIO.ConnectionConfiguration .create(zkAddress, SOLR_COLLECTION) .withBasicCredentials("solr", "wrongpassword"); - try (AuthorizedCloudSolrClient solrClient = connectionConfiguration.createClient()) { - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + try (AuthorizedSolrClient solrClient = connectionConfiguration.createClient()) { + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); } } @Test public void testSizes() throws Exception { - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); @@ -144,7 +143,7 @@ public void testSizes() throws Exception { assertThat( "Wrong estimated size bellow minimum", estimatedSize, - greaterThan(SolrIOTestUtils.AVERAGE_DOC_SIZE * NUM_DOCS)); + greaterThan(SolrIOTestUtils.MIN_DOC_SIZE * NUM_DOCS)); assertThat( "Wrong estimated size beyond maximum", estimatedSize, @@ -153,13 +152,13 @@ public void testSizes() throws Exception { @Test public void testRead() throws Exception { - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); PCollection output = pipeline.apply( SolrIO.read() .withConnectionConfiguration(connectionConfiguration) - .withBatchSize(100L)); + .withBatchSize(101L)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); pipeline.run(); @@ -167,13 +166,13 @@ public void testRead() throws Exception { @Test public void testReadWithQuery() throws Exception { - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); PCollection output = pipeline.apply( SolrIO.read() .withConnectionConfiguration(connectionConfiguration) - .withQuery("scientist:Einstein")); + .withQuery("scientist:Franklin")); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); pipeline.run(); @@ -187,10 +186,10 @@ public void testWrite() throws Exception { .apply(SolrIO.write().withConnectionConfiguration(connectionConfiguration)); pipeline.run(); - long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(solrClient); assertEquals(NUM_DOCS, currentNumDocs); - QueryResponse response = solrClient.query(SOLR_COLLECTION, new SolrQuery("scientist:Einstein")); + QueryResponse response = solrClient.query(new SolrQuery("scientist:Lovelace")); assertEquals(NUM_DOCS / NUM_SCIENTISTS, response.getResults().getNumFound()); } @@ -214,7 +213,7 @@ public void testWriteWithMaxBatchSize() throws Exception { // force the index to upgrade after inserting for the inserted docs // to be searchable immediately long currentNumDocs = SolrIOTestUtils - .commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + .commitAndGetCurrentNumDocs(solrClient); if ((numDocsProcessed % BATCH_SIZE) == 0) { /* bundle end */ assertEquals( @@ -238,7 +237,7 @@ public void testWriteWithMaxBatchSize() throws Exception { @Test public void testSplit() throws Exception { - SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); @@ -257,6 +256,8 @@ public void testSplit() throws Exception { nonEmptySplits += 1; } } + // docs are hashed by id to shards, in this test, NUM_DOCS >> NUM_SHARDS + // therefore, can not exist an empty shard. assertEquals("Wrong number of empty splits", expectedNumSplits, nonEmptySplits); } } diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java index 5582c2202c0b..cabf83c89c81 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -29,11 +29,11 @@ /** Test utilities to use with {@link SolrIO}. */ public class SolrIOTestUtils { - public static final long AVERAGE_DOC_SIZE = 25L; + public static final long MIN_DOC_SIZE = 25L; public static final long MAX_DOC_SIZE = 35L; static void createCollection(String collection, int numShards, int replicationFactor, - AuthorizedCloudSolrClient client) + AuthorizedSolrClient client) throws Exception { CollectionAdminRequest.Create create = new CollectionAdminRequest.Create() .setCollectionName(collection) @@ -44,22 +44,22 @@ static void createCollection(String collection, int numShards, int replicationFa } /** Inserts the given number of test documents into Solr. */ - static void insertTestDocuments(String collection, long numDocs, AuthorizedCloudSolrClient client) + static void insertTestDocuments(long numDocs, AuthorizedSolrClient client) throws IOException { List data = createDocuments(numDocs); try { UpdateRequest updateRequest = new UpdateRequest(); updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); updateRequest.add(data); - client.process(collection, updateRequest); + client.process(updateRequest); } catch (SolrServerException e) { - throw new IOException("Failed to insert test documents in collection " + collection, e); + throw new IOException("Failed to insert test documents to collection", e); } } /** Delete given collection. */ - static void deleteCollection(String collection, AuthorizedCloudSolrClient client) + static void deleteCollection(String collection, AuthorizedSolrClient client) throws IOException { try { CollectionAdminRequest.Delete delete = new CollectionAdminRequest.Delete() @@ -72,13 +72,13 @@ static void deleteCollection(String collection, AuthorizedCloudSolrClient client } /** Clear given collection. */ - static void clearCollection(String collection, AuthorizedCloudSolrClient client) + static void clearCollection(AuthorizedSolrClient client) throws IOException { try { UpdateRequest updateRequest = new UpdateRequest(); updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); updateRequest.deleteByQuery("*:*"); - client.process(collection, updateRequest); + client.process(updateRequest); } catch (SolrServerException e) { throw new IOException(e); } @@ -90,14 +90,14 @@ static void clearCollection(String collection, AuthorizedCloudSolrClient client) * * @return The number of docs in the index */ - static long commitAndGetCurrentNumDocs(String collection, AuthorizedCloudSolrClient client) + static long commitAndGetCurrentNumDocs(AuthorizedSolrClient client) throws IOException { SolrQuery solrQuery = new SolrQuery("*:*"); solrQuery.setRows(0); try { UpdateRequest update = new UpdateRequest(); update.setAction(UpdateRequest.ACTION.COMMIT, true, true); - client.process(collection, update); + client.process(update); return client.query(new SolrQuery("*:*")).getResults().getNumFound(); } catch (SolrServerException e) { @@ -112,10 +112,10 @@ static long commitAndGetCurrentNumDocs(String collection, AuthorizedCloudSolrCli */ static List createDocuments(long numDocs) { String[] scientists = { - "Einstein", - "Darwin", - "Copernicus", - "Pasteur", + "Lovelace", + "Franklin", + "Meitner", + "Hopper", "Curie", "Faraday", "Newton", From 90541d3af3949027802ab3508d4252e9cade2336 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Thu, 3 Aug 2017 09:26:17 +0700 Subject: [PATCH 10/14] SOLRIO: Fixing second round of review --- .../sdk/io/solr/AuthorizedSolrClient.java | 38 +++--- .../org/apache/beam/sdk/io/solr/SolrIO.java | 112 +++++++++++------- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 38 +++--- .../apache/beam/sdk/io/solr/SolrIOTest.java | 105 +++++++++------- .../beam/sdk/io/solr/SolrIOTestUtils.java | 18 +-- 5 files changed, 185 insertions(+), 126 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java index 74813ffe2cf5..44d7b88d9fde 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java @@ -28,45 +28,55 @@ import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.CoreAdminRequest; import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CoreAdminResponse; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.params.SolrParams; /** * Client for interact with Solr. - * The target replica/collection is pre-selected - * @param type of SolrClient + * @param type of SolrClient */ -class AuthorizedSolrClient implements Closeable { - private final T solrClient; +class AuthorizedSolrClient implements Closeable { + private final ClientT solrClient; private final String username; private final String password; - AuthorizedSolrClient(T solrClient, ConnectionConfiguration configuration) { + AuthorizedSolrClient(ClientT solrClient, ConnectionConfiguration configuration) { checkArgument( solrClient != null, - "AuthorizedSolrClient(solrClient, configuration) " - + "called with null solrClient"); + "solrClient can not be null"); checkArgument( configuration != null, - "AuthorizedSolrClient(solrClient, configuration) " - + "called with null configuration"); + "configuration can not be null"); this.solrClient = solrClient; this.username = configuration.getUsername(); this.password = configuration.getPassword(); } - QueryResponse query(SolrParams solrParams) + QueryResponse query(String collection, SolrParams solrParams) throws IOException, SolrServerException { QueryRequest query = new QueryRequest(solrParams); - return process(query); + return process(collection, query); } - T2 process(SolrRequest request) - throws IOException, SolrServerException { + ResponseT process(String collection, + SolrRequest request) throws IOException, SolrServerException { request.setBasicAuthCredentials(username, password); - return request.process(solrClient); + return request.process(solrClient, collection); + } + + CoreAdminResponse process(CoreAdminRequest request) + throws IOException, SolrServerException { + return process(null, request); + } + + SolrResponse process(CollectionAdminRequest request) + throws IOException, SolrServerException { + return process(null, request); } static ClusterState getClusterState( diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 50a5e68be6f3..8ac7013f5819 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -40,7 +40,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; @@ -62,8 +61,10 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.CoreAdminRequest; import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.client.solrj.request.schema.SchemaRequest; import org.apache.solr.client.solrj.response.CoreAdminResponse; import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.client.solrj.response.schema.SchemaResponse; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.ClusterState; @@ -117,18 +118,22 @@ * * } * - *

Optionally, you can provide {@code withBatchSize()} - * to specify the size of the write batch in number of documents. */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SolrIO { public static Read read() { - return new AutoValue_SolrIO_Read.Builder().setBatchSize(100L).setQuery("*:*").build(); + // 1000 for batch size is good enough in many cases, + // ex: if document size is large, around 10KB, the response's size will be around 10MB + // if document seize is small, around 1KB, the response's size will be around 1MB + return new AutoValue_SolrIO_Read.Builder().setBatchSize(1000).setQuery("*:*").build(); } public static Write write() { - return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000L).build(); + // 1000 for batch size is good enough in many cases, + // ex: if document size is large, around 10KB, the request's size will be around 10MB + // if document seize is small, around 1KB, the request's size will be around 1MB + return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000).build(); } private SolrIO() { @@ -139,7 +144,6 @@ private SolrIO() { public abstract static class ConnectionConfiguration implements Serializable { abstract String getZkHost(); - abstract String getCollection(); @Nullable abstract String getUsername(); @Nullable abstract String getPassword(); abstract Builder builder(); @@ -147,7 +151,6 @@ public abstract static class ConnectionConfiguration implements Serializable { @AutoValue.Builder abstract static class Builder { abstract Builder setZkHost(String zkHost); - abstract Builder setCollection(String collection); abstract Builder setUsername(String username); abstract Builder setPassword(String password); abstract ConnectionConfiguration build(); @@ -157,19 +160,14 @@ abstract static class Builder { * Creates a new Solr connection configuration. * * @param zkHost host of zookeeper - * @param collection the collection toward which the requests will be issued * @return the connection configuration object */ - public static ConnectionConfiguration create(String zkHost, String collection) { + public static ConnectionConfiguration create(String zkHost) { checkArgument(zkHost != null, "ConnectionConfiguration.create(zkHost, collection) " + "called with null address"); - checkArgument(collection != null, - "ConnectionConfiguration.create(zkHost, collection) " - + "called with null collectioin"); return new AutoValue_SolrIO_ConnectionConfiguration.Builder() .setZkHost(zkHost) - .setCollection(collection) .build(); } @@ -198,7 +196,6 @@ public ConnectionConfiguration withBasicCredentials(String username, String pass private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("zkHost", getZkHost())); - builder.add(DisplayData.item("collection", getCollection())); builder.addIfNotNull(DisplayData.item("username", getUsername())); } @@ -213,7 +210,6 @@ private HttpClient createHttpClient() { AuthorizedSolrClient createClient() throws MalformedURLException { CloudSolrClient solrClient = new CloudSolrClient(getZkHost(), createHttpClient()); - solrClient.setDefaultCollection(getCollection()); return new AuthorizedSolrClient<>(solrClient, this); } @@ -229,22 +225,22 @@ public abstract static class Read extends PTransform 0, "SolrIO.read().withBatchSize(batchSize) " + "called with a negative or equal to 0 value: %s", batchSize); checkArgument(batchSize <= MAX_BATCH_SIZE, @@ -296,6 +303,9 @@ public void validate(PipelineOptions options) { checkState(getConnectionConfiguration() != null, "SolrIO.read() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); + checkState(getCollection() != null, + "SolrIO.read() requires a collection name" + + " to be set via withCollection(collection)"); } @Override @@ -338,7 +348,7 @@ public List> split(long desiredBundleSizeB ConnectionConfiguration connectionConfig = spec.getConnectionConfiguration(); List sources = new ArrayList<>(); try (AuthorizedSolrClient client = connectionConfig.createClient()) { - String collection = connectionConfig.getCollection(); + String collection = spec.getCollection(); final ClusterState clusterState = AuthorizedSolrClient.getClusterState(client); DocCollection docCollection = clusterState.getCollection(collection); for (Slice slice : docCollection.getSlices()) { @@ -372,7 +382,7 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { } } - private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOException { + private long getEstimatedSizeOfShard(ReplicaInfo replica) throws IOException { try (AuthorizedSolrClient solrClient = spec.getConnectionConfiguration() .createClient(replica.baseUrl())) { CoreAdminRequest req = new CoreAdminRequest(); @@ -380,7 +390,7 @@ private long getEstimatedSizeOfShard(@Nonnull ReplicaInfo replica) throws IOExce req.setIndexInfoNeeded(true); CoreAdminResponse response; try { - response = (CoreAdminResponse) solrClient.process(req); + response = solrClient.process(req); } catch (SolrServerException e) { throw new IOException("Can not get core status from " + replica, e); } @@ -395,7 +405,7 @@ private long getEstimatedSizeOfCollection() throws IOException { ConnectionConfiguration config = spec.getConnectionConfiguration(); try (AuthorizedSolrClient solrClient = config.createClient()) { DocCollection docCollection = AuthorizedSolrClient.getClusterState(solrClient) - .getCollection(config.getCollection()); + .getCollection(spec.getCollection()); if (docCollection.getSlices().isEmpty()) { return 0; } @@ -475,6 +485,7 @@ private static class BoundedSolrReader extends BoundedSource.BoundedReader batchIterator; private boolean done; + private String uniqueKey; private BoundedSolrReader(BoundedSolrSource source) { this.source = source; @@ -485,20 +496,21 @@ private BoundedSolrReader(BoundedSolrSource source) { public boolean start() throws IOException { if (source.replica != null) { solrClient = source.spec.getConnectionConfiguration() - .createClient(source.replica.coreUrl()); + .createClient(source.replica.baseUrl()); } else { solrClient = source.spec.getConnectionConfiguration() .createClient(); } - - SolrQuery solrParams = getQueryParams(source); + SchemaRequest.UniqueKey uniqueKeyRequest = new SchemaRequest.UniqueKey(); try { - QueryResponse response = solrClient.query(solrParams); - updateCursorMark(response); - return readNextBatchAndReturnFirstDocument(response); + String collection = source.spec.getCollection(); + SchemaResponse.UniqueKeyResponse uniqueKeyResponse = + (SchemaResponse.UniqueKeyResponse) solrClient.process(collection, uniqueKeyRequest); + uniqueKey = uniqueKeyResponse.getUniqueKey(); } catch (SolrServerException e) { - throw new IOException(e); + throw new IOException("Can not get unique key from solr", e); } + return advance(); } private SolrQuery getQueryParams(BoundedSolrSource source) { @@ -508,7 +520,8 @@ private SolrQuery getQueryParams(BoundedSolrSource source) { } SolrQuery solrQuery = new SolrQuery(query); solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark); - solrQuery.addSort("id", SolrQuery.ORDER.asc); + solrQuery.setRows(source.spec.getBatchSize()); + solrQuery.addSort(uniqueKey, SolrQuery.ORDER.asc); if (source.replica != null) { solrQuery.setDistrib(false); } @@ -524,13 +537,18 @@ private void updateCursorMark(QueryResponse response) { @Override public boolean advance() throws IOException { - if (batchIterator.hasNext()) { + if (batchIterator != null && batchIterator.hasNext()) { current = batchIterator.next(); return true; } else { SolrQuery solrQuery = getQueryParams(source); try { - QueryResponse response = solrClient.query(solrQuery); + QueryResponse response; + if (source.replica != null) { + response = solrClient.query(source.replica.coreName(), solrQuery); + } else { + response = solrClient.query(source.spec.getCollection(), solrQuery); + } updateCursorMark(response); return readNextBatchAndReturnFirstDocument(response); } catch (SolrServerException e) { @@ -575,20 +593,20 @@ public BoundedSource getCurrentSource() { public abstract static class Write extends PTransform, PDone> { @Nullable abstract ConnectionConfiguration getConnectionConfiguration(); - abstract long getMaxBatchSize(); + @Nullable abstract String getCollection(); + abstract int getMaxBatchSize(); abstract Builder builder(); @AutoValue.Builder abstract static class Builder { abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); - abstract Builder setMaxBatchSize(long maxBatchSize); + abstract Builder setCollection(String collection); + abstract Builder setMaxBatchSize(int maxBatchSize); abstract Write build(); } /** * Provide the Solr connection configuration object. - * - * @param connectionConfiguration the Solr {@link ConnectionConfiguration} object */ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { checkArgument(connectionConfiguration != null, "SolrIO.write()" @@ -596,6 +614,16 @@ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfi return builder().setConnectionConfiguration(connectionConfiguration).build(); } + /** + * Provide name of collection while reading from Solr. + * + * @param collection the collection toward which the requests will be issued + */ + public Write withCollection(String collection) { + checkArgument(collection != null, "collection can not be null"); + return builder().setCollection(collection).build(); + } + /** * Provide a maximum size in number of documents for the batch. Depending on the * execution engine, size of bundles may vary, this sets the maximum size. Change this if you @@ -603,7 +631,8 @@ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfi * * @param batchSize maximum batch size in number of documents */ - public Write withMaxBatchSize(long batchSize) { + @VisibleForTesting + Write withMaxBatchSize(int batchSize) { checkArgument(batchSize > 0, "SolrIO.write()" + ".withMaxBatchSize(batchSize) called with incorrect <= 0 value"); return builder().setMaxBatchSize(batchSize).build(); @@ -614,6 +643,9 @@ public void validate(PipelineOptions options) { checkState(getConnectionConfiguration() != null, "SolrIO.write() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); + checkState(getCollection() != null, + "SolrIO.write() requires a collection name" + + " to be set via withCollection(collection)"); } @Override @@ -665,7 +697,7 @@ private void flushBatch() throws IOException { try { UpdateRequest updateRequest = new UpdateRequest(); updateRequest.add(batch); - solrClient.process(updateRequest); + solrClient.process(spec.getCollection(), updateRequest); } catch (SolrServerException e) { throw new IOException("Error writing to Solr", e); } finally { diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java index 419058220a3a..5de4ce110be8 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertEquals; import java.util.List; @@ -69,31 +70,32 @@ public class SolrIOIT { private static AuthorizedSolrClient solrClient; private static IOTestPipelineOptions options; - private static SolrIO.ConnectionConfiguration readConnectionConfiguration; + private static SolrIO.ConnectionConfiguration connectionConfiguration; @Rule public TestPipeline pipeline = TestPipeline.create(); @BeforeClass public static void beforeClass() throws Exception { PipelineOptionsFactory.register(IOTestPipelineOptions.class); options = TestPipeline.testingPipelineOptions().as(IOTestPipelineOptions.class); - readConnectionConfiguration = SolrIO.ConnectionConfiguration - .create(options.getSolrZookeeperServer(), READ_COLLECTION); - solrClient = readConnectionConfiguration.createClient(); + connectionConfiguration = SolrIO.ConnectionConfiguration + .create(options.getSolrZookeeperServer()); + solrClient = connectionConfiguration.createClient(); CollectionAdminResponse listResponse = (CollectionAdminResponse) solrClient.process(new CollectionAdminRequest.List()); List collections = (List) listResponse.getResponse().get("collections"); if (collections.contains(READ_COLLECTION)) { - QueryResponse response = solrClient.query(new SolrQuery("*:*")); + QueryResponse response = solrClient.query(READ_COLLECTION, new SolrQuery("*:*")); if (response.getResults().getNumFound() != NUM_DOCS) { LOG.info("Collection {} is exist but the number of documents is not match, repopulate", READ_COLLECTION); - SolrIOTestUtils.clearCollection(solrClient); - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.clearCollection(READ_COLLECTION, solrClient); + SolrIOTestUtils.insertTestDocuments(READ_COLLECTION, NUM_DOCS, solrClient); } } else { LOG.info("Create and populate collection {}", READ_COLLECTION); + SolrIOTestUtils.createCollection(READ_COLLECTION, NUM_SHARDS, 1, solrClient); } SolrIOTestUtils.createCollection(WRITE_COLLECTION, 2, 2, solrClient); @@ -108,7 +110,7 @@ public static void afterClass() throws Exception { @Test public void testSplitsVolume() throws Exception { SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.read().withConnectionConfiguration(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored now @@ -134,7 +136,7 @@ public void testSplitsVolume() throws Exception { public void testReadVolume() throws Exception { PCollection output = pipeline.apply( - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration)); + SolrIO.read().withConnectionConfiguration(connectionConfiguration)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); pipeline.run(); @@ -142,24 +144,24 @@ public void testReadVolume() throws Exception { @Test public void testWriteVolume() throws Exception { - SolrIO.ConnectionConfiguration writeConnectionConfiguration = - SolrIO.ConnectionConfiguration.create(options.getSolrZookeeperServer(), WRITE_COLLECTION); List data = SolrIOTestUtils.createDocuments(NUM_DOCS); pipeline .apply(Create.of(data)) - .apply(SolrIO.write().withConnectionConfiguration(writeConnectionConfiguration)); + .apply(SolrIO.write() + .withConnectionConfiguration(connectionConfiguration) + .withCollection(WRITE_COLLECTION) + ); pipeline.run(); - try (AuthorizedSolrClient solrClient = writeConnectionConfiguration.createClient()) { - long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(solrClient); - assertEquals(NUM_DOCS, currentNumDocs); - } + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(WRITE_COLLECTION, solrClient); + assertEquals(NUM_DOCS, currentNumDocs); + } @Test public void testEstimatedSizesVolume() throws Exception { SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(readConnectionConfiguration); + SolrIO.read().withConnectionConfiguration(connectionConfiguration); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size @@ -173,6 +175,6 @@ public void testEstimatedSizesVolume() throws Exception { assertThat( "Wrong estimated size beyond maximum", estimatedSize, - greaterThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); + lessThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); } } diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java index 434308b08215..ff1534e05f09 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; import com.google.common.io.BaseEncoding; @@ -68,7 +69,7 @@ public class SolrIOTest extends SolrCloudTestCase{ private static final int NUM_SHARDS = 3; private static final long NUM_DOCS = 400L; private static final int NUM_SCIENTISTS = 10; - private static final long BATCH_SIZE = 200L; + private static final int BATCH_SIZE = 200; private static AuthorizedSolrClient solrClient; private static SolrIO.ConnectionConfiguration connectionConfiguration; @@ -78,9 +79,12 @@ public class SolrIOTest extends SolrCloudTestCase{ @BeforeClass public static void beforeClass() throws Exception { - //setup credential for solr user + // setup credential for solr user, + // See https://cwiki.apache.org/confluence/display/solr/Basic+Authentication+Plugin String password = "SolrRocks"; - byte[] salt = new byte[32]; + // salt's size can be arbitrary + byte[] salt = new byte[random().nextInt(30) + 1]; + random().nextBytes(salt); String base64Salt = BaseEncoding.base64().encode(salt); String sha56 = Sha256AuthenticationProvider.sha256(password, base64Salt); String credential = sha56 + " " + base64Salt; @@ -98,7 +102,7 @@ public static void beforeClass() throws Exception { zkStateReader.getZkClient() .setData("/security.json", securityJson.getBytes(Charset.defaultCharset()), true); String zkAddress = cluster.getZkServer().getZkAddress(); - connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress, SOLR_COLLECTION) + connectionConfiguration = SolrIO.ConnectionConfiguration.create(zkAddress) .withBasicCredentials("solr", password); solrClient = connectionConfiguration.createClient(); SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, 1, solrClient); @@ -111,7 +115,7 @@ public static void afterClass() throws Exception { @Before public void before() throws Exception { - SolrIOTestUtils.clearCollection(solrClient); + SolrIOTestUtils.clearCollection(SOLR_COLLECTION, solrClient); } @Rule @@ -122,19 +126,21 @@ public void testBadCredentials() throws IOException { String zkAddress = cluster.getZkServer().getZkAddress(); SolrIO.ConnectionConfiguration connectionConfiguration = SolrIO.ConnectionConfiguration - .create(zkAddress, SOLR_COLLECTION) + .create(zkAddress) .withBasicCredentials("solr", "wrongpassword"); try (AuthorizedSolrClient solrClient = connectionConfiguration.createClient()) { - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); } } @Test public void testSizes() throws Exception { - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.Read read = SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .withCollection(SOLR_COLLECTION); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); // can't use equal assert as Solr collections never have same size // (due to internal Lucene implementation) @@ -147,18 +153,19 @@ public void testSizes() throws Exception { assertThat( "Wrong estimated size beyond maximum", estimatedSize, - greaterThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); + lessThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); } @Test public void testRead() throws Exception { - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PCollection output = pipeline.apply( SolrIO.read() .withConnectionConfiguration(connectionConfiguration) - .withBatchSize(101L)); + .withCollection(SOLR_COLLECTION) + .withBatchSize(101)); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS); pipeline.run(); @@ -166,12 +173,13 @@ public void testRead() throws Exception { @Test public void testReadWithQuery() throws Exception { - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PCollection output = pipeline.apply( SolrIO.read() .withConnectionConfiguration(connectionConfiguration) + .withCollection(SOLR_COLLECTION) .withQuery("scientist:Franklin")); PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); @@ -181,15 +189,19 @@ public void testReadWithQuery() throws Exception { @Test public void testWrite() throws Exception { List data = SolrIOTestUtils.createDocuments(NUM_DOCS); + SolrIO.Write write = + SolrIO.write() + .withConnectionConfiguration(connectionConfiguration) + .withCollection(SOLR_COLLECTION); pipeline .apply(Create.of(data)) - .apply(SolrIO.write().withConnectionConfiguration(connectionConfiguration)); + .apply(write); pipeline.run(); - long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(solrClient); + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); assertEquals(NUM_DOCS, currentNumDocs); - QueryResponse response = solrClient.query(new SolrQuery("scientist:Lovelace")); + QueryResponse response = solrClient.query(SOLR_COLLECTION, new SolrQuery("scientist:Lovelace")); assertEquals(NUM_DOCS / NUM_SCIENTISTS, response.getResults().getNumFound()); } @@ -198,49 +210,52 @@ public void testWriteWithMaxBatchSize() throws Exception { SolrIO.Write write = SolrIO.write() .withConnectionConfiguration(connectionConfiguration) + .withCollection(SOLR_COLLECTION) .withMaxBatchSize(BATCH_SIZE); // write bundles size is the runner decision, we cannot force a bundle size, // so we test the Writer as a DoFn outside of a runner. - DoFnTester fnTester = DoFnTester.of(new SolrIO.Write.WriteFn(write)); - List input = SolrIOTestUtils.createDocuments(NUM_DOCS); - long numDocsProcessed = 0; - long numDocsInserted = 0; - for (SolrInputDocument document : input) { - fnTester.processElement(document); - numDocsProcessed++; - // test every 100 docs to avoid overloading Solr - if ((numDocsProcessed % 100) == 0) { - // force the index to upgrade after inserting for the inserted docs - // to be searchable immediately - long currentNumDocs = SolrIOTestUtils - .commitAndGetCurrentNumDocs(solrClient); - if ((numDocsProcessed % BATCH_SIZE) == 0) { + try (DoFnTester fnTester = + DoFnTester.of(new SolrIO.Write.WriteFn(write))) { + List input = SolrIOTestUtils.createDocuments(NUM_DOCS); + long numDocsProcessed = 0; + long numDocsInserted = 0; + for (SolrInputDocument document : input) { + fnTester.processElement(document); + numDocsProcessed++; + // test every 100 docs to avoid overloading Solr + if ((numDocsProcessed % 100) == 0) { + // force the index to upgrade after inserting for the inserted docs + // to be searchable immediately + long currentNumDocs = SolrIOTestUtils + .commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + if ((numDocsProcessed % BATCH_SIZE) == 0) { /* bundle end */ - assertEquals( - "we are at the end of a bundle, we should have inserted all processed documents", - numDocsProcessed, - currentNumDocs); - numDocsInserted = currentNumDocs; - } else { + assertEquals( + "we are at the end of a bundle, we should have inserted all processed documents", + numDocsProcessed, + currentNumDocs); + numDocsInserted = currentNumDocs; + } else { /* not bundle end */ - assertEquals( - "we are not at the end of a bundle, we should have inserted no more documents", - numDocsInserted, - currentNumDocs); - } + assertEquals( + "we are not at the end of a bundle, we should have inserted no more documents", + numDocsInserted, + currentNumDocs); + } + } } } - fnTester.finishBundle(); - fnTester.close(); } @Test public void testSplit() throws Exception { - SolrIOTestUtils.insertTestDocuments(NUM_DOCS, solrClient); + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); PipelineOptions options = PipelineOptionsFactory.create(); - SolrIO.Read read = SolrIO.read().withConnectionConfiguration(connectionConfiguration); + SolrIO.Read read = SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .withCollection(SOLR_COLLECTION); SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); //desiredBundleSize is ignored for now int desiredBundleSizeBytes = 0; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java index cabf83c89c81..234382b1957a 100644 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -29,8 +29,8 @@ /** Test utilities to use with {@link SolrIO}. */ public class SolrIOTestUtils { - public static final long MIN_DOC_SIZE = 25L; - public static final long MAX_DOC_SIZE = 35L; + public static final long MIN_DOC_SIZE = 40L; + public static final long MAX_DOC_SIZE = 90L; static void createCollection(String collection, int numShards, int replicationFactor, AuthorizedSolrClient client) @@ -44,14 +44,14 @@ static void createCollection(String collection, int numShards, int replicationFa } /** Inserts the given number of test documents into Solr. */ - static void insertTestDocuments(long numDocs, AuthorizedSolrClient client) + static void insertTestDocuments(String collection, long numDocs, AuthorizedSolrClient client) throws IOException { List data = createDocuments(numDocs); try { UpdateRequest updateRequest = new UpdateRequest(); updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); updateRequest.add(data); - client.process(updateRequest); + client.process(collection, updateRequest); } catch (SolrServerException e) { throw new IOException("Failed to insert test documents to collection", e); } @@ -72,13 +72,13 @@ static void deleteCollection(String collection, AuthorizedSolrClient client) } /** Clear given collection. */ - static void clearCollection(AuthorizedSolrClient client) + static void clearCollection(String collection, AuthorizedSolrClient client) throws IOException { try { UpdateRequest updateRequest = new UpdateRequest(); updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); updateRequest.deleteByQuery("*:*"); - client.process(updateRequest); + client.process(collection, updateRequest); } catch (SolrServerException e) { throw new IOException(e); } @@ -90,16 +90,16 @@ static void clearCollection(AuthorizedSolrClient client) * * @return The number of docs in the index */ - static long commitAndGetCurrentNumDocs(AuthorizedSolrClient client) + static long commitAndGetCurrentNumDocs(String collection, AuthorizedSolrClient client) throws IOException { SolrQuery solrQuery = new SolrQuery("*:*"); solrQuery.setRows(0); try { UpdateRequest update = new UpdateRequest(); update.setAction(UpdateRequest.ACTION.COMMIT, true, true); - client.process(update); + client.process(collection, update); - return client.query(new SolrQuery("*:*")).getResults().getNumFound(); + return client.query(collection, new SolrQuery("*:*")).getResults().getNumFound(); } catch (SolrServerException e) { throw new IOException(e); } From 25f2801e270ddfda50f5510ff88ed683e6229d54 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 4 Aug 2017 06:08:50 +0700 Subject: [PATCH 11/14] Remove SolrIOIT --- .../org/apache/beam/sdk/io/solr/SolrIOIT.java | 180 ------------------ 1 file changed, 180 deletions(-) delete mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java deleted file mode 100644 index 5de4ce110be8..000000000000 --- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOIT.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solr; - -import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.common.IOTestPipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.SourceTestUtils; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; -import org.apache.solr.client.solrj.SolrQuery; -import org.apache.solr.client.solrj.request.CollectionAdminRequest; -import org.apache.solr.client.solrj.response.CollectionAdminResponse; -import org.apache.solr.client.solrj.response.QueryResponse; -import org.apache.solr.common.SolrDocument; -import org.apache.solr.common.SolrInputDocument; -import org.joda.time.Instant; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A test of {@link SolrIO} on an independent Solr instance. - * - *

This test requires a running instance of Solr (./bin/solr start -e cloud -noprompt) - * - *

You can run this test by doing the following from the beam parent module directory: - * - *

- *  mvn -e -Pio-it verify -pl sdks/java/io/solr -DintegrationTestPipelineOptions='[
- *  "--solrZookeeperServer=127.0.0.1:9983"]'
- * 
- */ -public class SolrIOIT { - - public static final String READ_COLLECTION = "beam"; - public static final int NUM_SHARDS = 3; - public static final long NUM_DOCS = 60000; - public static final String WRITE_COLLECTION = "beam" + Instant.now().getMillis(); - private static final Logger LOG = LoggerFactory.getLogger(SolrIOIT.class); - - private static AuthorizedSolrClient solrClient; - private static IOTestPipelineOptions options; - private static SolrIO.ConnectionConfiguration connectionConfiguration; - @Rule public TestPipeline pipeline = TestPipeline.create(); - - @BeforeClass - public static void beforeClass() throws Exception { - PipelineOptionsFactory.register(IOTestPipelineOptions.class); - options = TestPipeline.testingPipelineOptions().as(IOTestPipelineOptions.class); - connectionConfiguration = SolrIO.ConnectionConfiguration - .create(options.getSolrZookeeperServer()); - solrClient = connectionConfiguration.createClient(); - - CollectionAdminResponse listResponse = - (CollectionAdminResponse) solrClient.process(new CollectionAdminRequest.List()); - List collections = (List) listResponse.getResponse().get("collections"); - if (collections.contains(READ_COLLECTION)) { - QueryResponse response = solrClient.query(READ_COLLECTION, new SolrQuery("*:*")); - if (response.getResults().getNumFound() != NUM_DOCS) { - LOG.info("Collection {} is exist but the number of documents is not match, repopulate", - READ_COLLECTION); - SolrIOTestUtils.clearCollection(READ_COLLECTION, solrClient); - SolrIOTestUtils.insertTestDocuments(READ_COLLECTION, NUM_DOCS, solrClient); - } - } else { - LOG.info("Create and populate collection {}", - READ_COLLECTION); - SolrIOTestUtils.createCollection(READ_COLLECTION, NUM_SHARDS, 1, solrClient); - } - - SolrIOTestUtils.createCollection(WRITE_COLLECTION, 2, 2, solrClient); - } - - @AfterClass - public static void afterClass() throws Exception { - SolrIOTestUtils.deleteCollection(WRITE_COLLECTION, solrClient); - solrClient.close(); - } - - @Test - public void testSplitsVolume() throws Exception { - SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(connectionConfiguration); - SolrIO.BoundedSolrSource initialSource = - new SolrIO.BoundedSolrSource(read, null); - //desiredBundleSize is ignored now - long desiredBundleSizeBytes = 0; - List> splits = - initialSource.split(desiredBundleSizeBytes, options); - SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, options); - //this is the number of Solr shards - long expectedNumSplits = NUM_SHARDS; - assertEquals(expectedNumSplits, splits.size()); - int nonEmptySplits = 0; - for (BoundedSource subSource : splits) { - if (readFromSource(subSource, options).size() > 0) { - nonEmptySplits += 1; - } - } - // docs are hashed by id to shards, in this test, NUM_DOCS >> NUM_SHARDS - // therefore, can not exist an empty shard. - assertEquals(expectedNumSplits, nonEmptySplits); - } - - @Test - public void testReadVolume() throws Exception { - PCollection output = - pipeline.apply( - SolrIO.read().withConnectionConfiguration(connectionConfiguration)); - PAssert.thatSingleton(output.apply("Count", Count.globally())) - .isEqualTo(NUM_DOCS); - pipeline.run(); - } - - @Test - public void testWriteVolume() throws Exception { - List data = SolrIOTestUtils.createDocuments(NUM_DOCS); - pipeline - .apply(Create.of(data)) - .apply(SolrIO.write() - .withConnectionConfiguration(connectionConfiguration) - .withCollection(WRITE_COLLECTION) - ); - pipeline.run(); - - long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(WRITE_COLLECTION, solrClient); - assertEquals(NUM_DOCS, currentNumDocs); - - } - - @Test - public void testEstimatedSizesVolume() throws Exception { - SolrIO.Read read = - SolrIO.read().withConnectionConfiguration(connectionConfiguration); - SolrIO.BoundedSolrSource initialSource = - new SolrIO.BoundedSolrSource(read, null); - // can't use equal assert as Solr collections never have same size - // (due to internal Lucene implementation) - long estimatedSize = initialSource.getEstimatedSizeBytes(options); - LOG.info("Estimated size: {}", estimatedSize); - assertThat( - "Wrong estimated size bellow minimum", - estimatedSize, - greaterThan(SolrIOTestUtils.MIN_DOC_SIZE * NUM_DOCS)); - assertThat( - "Wrong estimated size beyond maximum", - estimatedSize, - lessThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); - } -} From 4704c7aec7ebf1ea20765f4ddac89f916e27cac2 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 4 Aug 2017 06:09:20 +0700 Subject: [PATCH 12/14] Add TODO for SolrIO.Read --- .../solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 8ac7013f5819..9b6b2619a9fc 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -365,6 +365,8 @@ public List> split(long desiredBundleSizeB break; } } + // TODO in case of this replica goes inactive while the pipeline runs. + // We should pick another active replica of this shard. checkState(randomActiveReplica != null, "Can not found an active replica for slice %s", slice.getName()); sources.add(new BoundedSolrSource(spec, randomActiveReplica)); From f38ed6fafb89c91a923b96c0580262704c9c8ded Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 4 Aug 2017 06:24:11 +0700 Subject: [PATCH 13/14] Make messages of checkArgument more consistent --- .../org/apache/beam/sdk/io/solr/SolrIO.java | 49 +++++++------------ 1 file changed, 17 insertions(+), 32 deletions(-) diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java index 9b6b2619a9fc..23ea98d72698 100644 --- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -22,7 +22,6 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Strings; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -163,9 +162,7 @@ abstract static class Builder { * @return the connection configuration object */ public static ConnectionConfiguration create(String zkHost) { - checkArgument(zkHost != null, - "ConnectionConfiguration.create(zkHost, collection) " - + "called with null address"); + checkArgument(zkHost != null, "zkHost can not be null"); return new AutoValue_SolrIO_ConnectionConfiguration.Builder() .setZkHost(zkHost) .build(); @@ -175,22 +172,10 @@ public static ConnectionConfiguration create(String zkHost) { * If Solr basic authentication is enabled, provide the username and password. */ public ConnectionConfiguration withBasicCredentials(String username, String password) { - checkArgument( - username != null, - "ConnectionConfiguration.create().withBasicCredentials(username, password) " - + "called with null username"); - checkArgument( - !username.isEmpty(), - "ConnectionConfiguration.create().withBasicCredentials(username, password) " - + "called with empty username"); - checkArgument( - password != null, - "ConnectionConfiguration.create().withBasicCredentials(username, password) " - + "called with null username"); - checkArgument( - !password.isEmpty(), - "ConnectionConfiguration.create().withBasicCredentials(username, password) " - + "called with empty username"); + checkArgument(username != null, "username can not be null"); + checkArgument(!username.isEmpty(), "username can not be empty"); + checkArgument(password != null, "password can not be null"); + checkArgument(!password.isEmpty(), "password can not be empty"); return builder().setUsername(username).setPassword(password).build(); } @@ -243,8 +228,8 @@ abstract static class Builder { * Provide the Solr connection configuration object. */ public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { - checkArgument(connectionConfiguration != null, "SolrIO.read()" - + ".withConnectionConfiguration(configuration) called with null configuration"); + checkArgument(connectionConfiguration != null, + "connectionConfiguration can not be null"); return builder().setConnectionConfiguration(connectionConfiguration).build(); } @@ -267,8 +252,8 @@ public Read withCollection(String collection) { * */ public Read withQuery(String query) { - checkArgument(!Strings.isNullOrEmpty(query), - "SolrIO.read().withQuery(query) called" + " with null or empty query"); + checkArgument(query != null, "query can not be null"); + checkArgument(!query.isEmpty(), "query can not be empty"); return builder().setQuery(query).build(); } @@ -283,11 +268,10 @@ public Read withQuery(String query) { */ @VisibleForTesting Read withBatchSize(int batchSize) { - checkArgument(batchSize > 0, "SolrIO.read().withBatchSize(batchSize) " - + "called with a negative or equal to 0 value: %s", batchSize); - checkArgument(batchSize <= MAX_BATCH_SIZE, - "SolrIO.read().withBatchSize(batchSize) " - + "called with a too large value (over %s): %s", + // TODO remove this configuration, we can figure out the best number + // by tuning batchSize when pipelines run. + checkArgument(batchSize > 0 && batchSize < MAX_BATCH_SIZE, + "Valid values for batchSize are 1 (inclusize) to %s (exclusive), but was: %s ", MAX_BATCH_SIZE, batchSize); return builder().setBatchSize(batchSize).build(); } @@ -611,8 +595,7 @@ abstract static class Builder { * Provide the Solr connection configuration object. */ public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { - checkArgument(connectionConfiguration != null, "SolrIO.write()" - + ".withConnectionConfiguration(configuration) called with null configuration"); + checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null"); return builder().setConnectionConfiguration(connectionConfiguration).build(); } @@ -635,8 +618,10 @@ public Write withCollection(String collection) { */ @VisibleForTesting Write withMaxBatchSize(int batchSize) { + // TODO remove this configuration, we can figure out the best number + // by tuning batchSize when pipelines run. checkArgument(batchSize > 0, - "SolrIO.write()" + ".withMaxBatchSize(batchSize) called with incorrect <= 0 value"); + "batchSize must be larger than 0, but was: %s", batchSize); return builder().setMaxBatchSize(batchSize).build(); } From f0cc0498d442edb8975c99c6a77a9bd5a0c1c387 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Fri, 4 Aug 2017 06:31:10 +0700 Subject: [PATCH 14/14] Remove unused dependencies --- sdks/java/io/solr/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml index fa112add90b7..d269f7911f34 100644 --- a/sdks/java/io/solr/pom.xml +++ b/sdks/java/io/solr/pom.xml @@ -52,11 +52,6 @@ jsr305
- - joda-time - joda-time - - org.apache.commons commons-compress