You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by hmcl <gi...@git.apache.org> on 2015/08/04 07:27:39 UTC

[GitHub] storm pull request: STORM-851: Storm Solr Connector

GitHub user hmcl opened a pull request:

    https://github.com/apache/storm/pull/665

    STORM-851: Storm Solr Connector

    1. SolrUpdate Bolt
    2. Trident State implementation
    3. Fields Mapper
    4. JSON Mapper
    5. Integration Tests

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hmcl/storm-apache STORM-851

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/665.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #665
    
----
commit 0c5e00caf801e8a7455f0bc7976a7dcd0f8ab335
Author: Hugo Louro <hm...@gmail.com>
Date:   2015-07-17T02:10:30Z

    STORM-851: Storm Solr Connector
    1. SolrUpdate Bolt
    2. Trident State implementation
    3. Fields Mapper
    4. JSON Mapper
    5. Integration Tests

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36992356
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/mapper/SolrJsonMapper.java ---
    @@ -0,0 +1,97 @@
    +package org.apache.storm.solr.mapper;
    +
    +import backtype.storm.tuple.ITuple;
    +import com.google.gson.Gson;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.List;
    +
    +/**
    + * Created by hlouro on 7/24/15.
    + */
    +public class SolrJsonMapper implements SolrMapper {
    +    private static final Logger logger = LoggerFactory.getLogger(SolrJsonMapper.class);
    +    private static final String JSON_UPDATE_URL = "/update/json/docs";
    --- End diff --
    
    should we expose this as configurable option?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36990357
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java ---
    @@ -0,0 +1,107 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.apache.storm.solr.config.SolrCommitStrategy;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.apache.storm.solr.mapper.SolrMapper;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/19/15.
    + */
    +public class SolrUpdateBolt extends AbstractSolrBolt {
    +    private final Logger logger = LoggerFactory.getLogger(SolrUpdateBolt.class);
    +    private final SolrMapper solrMapper;
    +    private final SolrCommitStrategy commitStgy;
    +    private List<Tuple> toCommitTuples;
    +    private final String ackFailLock = "LOCK";      //serializable lock
    +
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper) {
    +        this(solrConfig, solrMapper, null);
    +    }
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper, SolrCommitStrategy commitStgy) {
    +        super(solrConfig);
    +        this.solrMapper = solrMapper;
    +        this.commitStgy = commitStgy;
    +        logger.info("Created {} with the following configuration: " +
    +                    "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = {}]",
    +                    this.getClass().getSimpleName(), solrConfig, solrMapper, commitStgy);
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        super.prepare(stormConf, context, collector);
    +        this.toCommitTuples = new LinkedList<>();
    +    }
    +
    +    @Override
    +    public void execute(Tuple tuple) {
    +        try {
    +            SolrRequest request = solrMapper.toSolrRequest(tuple);
    +            solrClient.request(request, solrMapper.getCollection());
    +            ack(tuple);
    +        } catch (Exception e) {
    +            fail(tuple, e);
    +        }
    +    }
    +
    +    private void ack(Tuple tuple) throws SolrServerException, IOException {
    +        if (commitStgy == null) {
    +            collector.ack(tuple);
    +        } else {
    +            synchronized(ackFailLock) {
    --- End diff --
    
    don't think we need a synchronized block for this. Since there won't be any other threads trying modify toCommitTupes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-130751200
  
    I made a initial pass on the PR. Overall it looks good to me. It needs README to document on how to use Bolt and TridentState.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-134693551
  
    I left a few minor comments. There are a few .gitignore files that can be removed, and the documentation should use the Maven shade plugin, instead of the assembly plugin.
    
    I'm +1 once those are addressed, and am also willing to sponsor this module.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928671
  
    --- Diff: pom.xml ---
    @@ -169,8 +169,9 @@
             <module>external/storm-redis</module>
             <module>external/storm-eventhubs</module>
             <module>external/flux</module>
    -        <module>examples/storm-starter</module>
             <module>external/storm-elasticsearch</module>
    +        <module>external/storm-solr</module>
    +		<module>examples/storm-starter</module>
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928692
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36998880
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/schema/builder/RestJsonSchemaBuilder.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.storm.solr.schema.builder;
    +
    +import com.google.gson.Gson;
    +import org.apache.storm.solr.schema.Schema;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.Scanner;
    +
    +/**
    + * Class that buils the {@link Schema} object from the JSON representation of the schema as returned by the
    + * URL of the form http://localhost:8983/solr/gettingstarted/schema/ . This particular URL returns the schema
    + * in JSON format for the gettingstarted example running locally.
    + * <p></p>
    + * Created by hlouro on 7/28/15.
    + */
    +public class RestJsonSchemaBuilder implements SchemaBuilder {
    +    private static final Logger logger = LoggerFactory.getLogger(RestJsonSchemaBuilder.class);
    +    private Schema schema;
    +
    +
    +    /** Urls with the form http://localhost:8983/solr/gettingstarted/schema/ returns the schema in JSON format */
    +    public RestJsonSchemaBuilder(String solrHost, String solrPort, String collection) throws IOException {
    +        this(new URL("http://" + solrHost + ":" + solrPort + "/solr/" + collection + "/schema/"));
    --- End diff --
    
    The class has two other constructors that can receive the complete url either has a String or has a URL object. I believe this covers the arbitrary URL you mention. My idea with the constructor that receives the host, port, and collection is to be the default, if you will. I I am not sure /solr/ and /schema/ can be renamed on Solr. I can research that. In case they are, the option is to provide a 5 args constructor that receives these two in addition to the existing 3.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37883137
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/mapper/SolrJsonMapper.java ---
    @@ -0,0 +1,97 @@
    +package org.apache.storm.solr.mapper;
    +
    +import backtype.storm.tuple.ITuple;
    +import com.google.gson.Gson;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.List;
    +
    +/**
    + * Created by hlouro on 7/24/15.
    + */
    +public class SolrJsonMapper implements SolrMapper {
    +    private static final Logger logger = LoggerFactory.getLogger(SolrJsonMapper.class);
    +    private static final String JSON_UPDATE_URL = "/update/json/docs";
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928677
  
    --- Diff: external/storm-solr/src/test/java/org/apache/storm/solr/topology/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-134657971
  
    +1 .  @Parth-Brahmbhatt @ptgoetz  can you take a look at it as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37897758
  
    --- Diff: external/storm-solr/README.md ---
    @@ -0,0 +1,188 @@
    +# Storm Solr
    +Storm and Trident integration for Apache Solr. This package includes a bolt and a trident state that enable a Storm topology
    +stream the contents of storm tuples to index Solr collections.
    + 
    +# Index Storm tuples into a Solr collection
    +The The bolt and trident state provided use one of the supplied mappers to build a `SolrRequest` object that is 
    +responsible for making the update calls to Solr, thus updating the index of the collection specified.
    + 
    +# Usage Examples 
    +In this section we provide some simple code snippets on how to build Storm and Trident topologies to index Solr. In subsequent sections we 
    +describe in detail the two key components of the Storm Solr integration, the `SolrUpdateBolt`, and the `Mappers`, `SolrFieldsMapper`, and `SolrJsonMapper`.
    +
    +## Storm Bolt With JSON Mapper and Count Based Commit Strategy
    +
    +```java
    +    new SolrUpdateBolt(solrConfig, solrMapper, solrCommitStgy)
    +    
    +    // zkHostString for Solr 'gettingstarted' example
    +    SolrConfig solrConfig = new SolrConfig("127.0.0.1:9983");
    +    
    +    // JSON Mapper used to generate 'SolrRequest' requests to update the "gettingstarted" Solr collection with JSON content declared the tuple field with name "JSON"
    +    SolrMapper solrMapper = new SolrJsonMapper.Builder("gettingstarted", "JSON").build(); 
    +     
    +    // Acks every other five tuples. Setting to null acks every tuple
    +    SolrCommitStrategy solrCommitStgy = new CountBasedCommit(5);          
    +```
    +
    +## Trident Topology With Fields Mapper
    +```java
    +    new SolrStateFactory(solrConfig, solrMapper);
    +    
    +    // zkHostString for Solr 'gettingstarted' example
    +    SolrConfig solrConfig = new SolrConfig("127.0.0.1:9983");
    +    
    +    /* Solr Fields Mapper used to generate 'SolrRequest' requests to update the "gettingstarted" Solr collection. The Solr index is updated using the field values of the tuple fields that match static or dynamic fields declared in the schema object build using schemaBuilder */ 
    +    SolrMapper solrMapper = new SolrFieldsMapper.Builder(schemaBuilder, 'gettingstarted').build();
    +
    +    // builds the Schema object from the JSON representation of the schema as returned by the URL http://localhost:8983/solr/gettingstarted/schema/ 
    +    SchemaBuilder schemaBuilder = new RestJsonSchemaBuilder("localhost", "8983", "gettingstarted")
    +```
    +
    +## SolrUpdateBolt
    + `SolrUpdateBolt` streams tuples directly into Apache Solr. The Solr index is updated using`SolrRequest` requests. 
    + The `SolrUpdateBolt` is configurable using implementations of `SolrConfig`, `SolrMapper`, and optionally `SolrCommitStrategy`.
    +   
    + The data to stream onto Solr is extracted from the tuples using the strategy defined in the `SolrMapper` implementation.
    + 
    + The `SolrRquest` can be sent every tuple, or according to a strategy defined by `SolrCommitStrategy` implementations. 
    + If a `SolrCommitStrategy` is in place and one of the tuples in the batch fails, the batch is not committed, and all the tuples in that 
    +  batch are marked as Fail, and retried. On the other hand, if all tuples succeed, the `SolrRequest` is committed and all tuples are successfully acked.
    + 
    + `SolrConfig` is the class containing Solr configuration to be made available to Storm Solr bolts. Any configuration needed in the bolts should be put in this class.
    + 
    +
    +## SolrMapper
    +`SorlMapper` implementations define the strategy to extract information from the tuples. The public method
    +`toSolrRequest` receives a tuple or a list of tuples and returns a `SolrRequest` object that is used to update the Solr index.
    +
    +
    +### SolrJsonMapper
    +The `SolrJsonMapper` creates a Solr update request that is sent to the URL endpoint defined by Solr as the resource 
    +destination for requests in JSON format. 
    + 
    +To create a `SolrJsonMapper` the client must specify the name of the collection to update as well as the 
    +tuple field that contains the JSON object used to update the Solr index. If the tuple does not contain the field specified, 
    +a `SolrMapperException` will be thrown when the method `toSolrRequest`is called. If the field exists, its value can either 
    +be a String with the contents in JSON format, or a Java object that will be serialized to JSON
    + 
    +Code snippet illustrating how to create a `SolrJsonMapper` object to update the `gettingstarted` Solr collection with JSON content 
    +declared in the tuple field with name "JSON"
    +``` java
    +    SolrMapper solrMapper = new SolrJsonMapper.Builder("gettingstarted", "JSON").build();
    +```
    +
    +
    +### SolrFieldsMapper
    +The `SolrFieldsMapper` creates a Solr update request that is sent to the Solr URL endpoint that handles the updates of `SolrInputDocument` objects.
    +
    +To create a `SolrFieldsMapper` the client must specify the name of the collection to update as well as the `SolrSchemaBuilder`. 
    +The Solr `Schema` is used to extract information about the Solr schema fields and corresponding types. This metadata is used
    +to get the information from the tuples. Only tuple fields that match a static or dynamic Solr fields are added to the document. Tuple fields 
    +that do not match the schema are not added to the `SolrInputDocument` being prepared for indexing. A debug log message is printed for the 
    + tuple fields that do not match the schema and hence are not indexed.
    + 
    +
    +The `SolrFieldsMapper` supports multivalue fields. A multivalue tuple field must be tokenized. The default token is |. Any 
    +arbitrary token can be specified by calling the method `org.apache.storm.solr.mapper.SolrFieldsMapper.Builder.setMultiValueFieldToken`
    +that is part of the `SolrFieldsMapper.Builder` builder class. 
    +
    +Code snippet illustrating how to create a `SolrFieldsMapper` object to update the `gettingstarted` Solr collection. The multivalue 
    +field separates each value with the token % instead of the default | . To use the default token you can ommit the call to the method
    +`setMultiValueFieldToken`.
    +
    +``` java
    +    new SolrFieldsMapper.Builder(
    +            new RestJsonSchemaBuilder("localhost", "8983", `gettingstarted`), `gettingstarted`)
    +                .setMultiValueFieldToken("%").build();
    +```
    +
    +# Build And Run Bundled Examples  
    +To be able to run the examples you must first build the java code in the package `storm-solr`, 
    +and then generate an uber jar with all the dependencies.
    +
    +
    +## Build the Storm Apache Solr Integration Code
    +
    +`mvn clean install -f REPO_HOME/storm/external/storm-solr`
    + 
    +## Use the Maven Assembly Plugin to Build the Uber Jar
    --- End diff --
    
    We should encourage the use of the Maven shade plugin over the assembly plugin. The assembly plugin doesn't handle jar manifest data properly and can cause problems with Hadoop dependencies.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928682
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/mapper/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37882952
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/mapper/SolrJsonMapper.java ---
    @@ -0,0 +1,97 @@
    +package org.apache.storm.solr.mapper;
    +
    +import backtype.storm.tuple.ITuple;
    +import com.google.gson.Gson;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.List;
    +
    +/**
    + * Created by hlouro on 7/24/15.
    + */
    +public class SolrJsonMapper implements SolrMapper {
    +    private static final Logger logger = LoggerFactory.getLogger(SolrJsonMapper.class);
    +    private static final String JSON_UPDATE_URL = "/update/json/docs";
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36993719
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/schema/builder/RestJsonSchemaBuilder.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.storm.solr.schema.builder;
    +
    +import com.google.gson.Gson;
    +import org.apache.storm.solr.schema.Schema;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.Scanner;
    +
    +/**
    + * Class that buils the {@link Schema} object from the JSON representation of the schema as returned by the
    + * URL of the form http://localhost:8983/solr/gettingstarted/schema/ . This particular URL returns the schema
    + * in JSON format for the gettingstarted example running locally.
    + * <p></p>
    + * Created by hlouro on 7/28/15.
    + */
    +public class RestJsonSchemaBuilder implements SchemaBuilder {
    +    private static final Logger logger = LoggerFactory.getLogger(RestJsonSchemaBuilder.class);
    +    private Schema schema;
    +
    +
    +    /** Urls with the form http://localhost:8983/solr/gettingstarted/schema/ returns the schema in JSON format */
    +    public RestJsonSchemaBuilder(String solrHost, String solrPort, String collection) throws IOException {
    +        this(new URL("http://" + solrHost + ":" + solrPort + "/solr/" + collection + "/schema/"));
    --- End diff --
    
    Does solr gives config options to rename "/solr/" part and also "/schema/" If so we shouldn't be hard coding those here instead give as an option with defaults.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928717
  
    --- Diff: external/storm-solr/pom.xml ---
    @@ -0,0 +1,106 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<project xmlns="http://maven.apache.org/POM/4.0.0"
    +         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <artifactId>storm</artifactId>
    +        <groupId>org.apache.storm</groupId>
    +        <version>0.11.0-SNAPSHOT</version>
    +        <relativePath>../../pom.xml</relativePath>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +
    +    <artifactId>storm-solr</artifactId>
    +
    +    <developers>
    +        <developer>
    +            <id>Hugo-Louro</id>
    +            <name>Hugo Louro</name>
    +            <email>hmclouro@gmail.com</email>
    +        </developer>
    +    </developers>
    +
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.storm</groupId>
    +            <artifactId>storm-core</artifactId>
    +            <version>${project.version}</version>
    +            <scope>provided</scope>
    +        </dependency>
    +        <!-- Solr and its dependencies -->
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-solrj</artifactId>
    +            <version>5.2.1</version>
    +            <scope>compile</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-codec</groupId>
    +            <artifactId>commons-codec</artifactId>
    +            <version>1.3</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-httpclient</groupId>
    +            <artifactId>commons-httpclient</artifactId>
    +            <version>3.1</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-io</groupId>
    +            <artifactId>commons-io</artifactId>
    +            <version>1.4</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-core</artifactId>
    +            <version>5.2.1</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-test-framework</artifactId>
    +            <version>5.2.1</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>com.google.guava</groupId>
    +            <artifactId>guava</artifactId>
    +            <version>17.0</version>
    +        </dependency>
    +        <!--test dependencies -->
    +        <dependency>
    +            <groupId>junit</groupId>
    +            <artifactId>junit</artifactId>
    +            <version>4.11</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>com.google.code.gson</groupId>
    +            <artifactId>gson</artifactId>
    +            <version>2.3.1</version>
    +        </dependency>
    +    </dependencies>
    +    <build>
    +        <plugins>
    +            <plugin>
    +                <groupId>org.apache.maven.plugins</groupId>
    +                <artifactId>maven-jar-plugin</artifactId>
    +                <version>2.5</version>
    +                <executions>
    +                    <execution>
    +                        <goals>
    +                            <goal>test-jar</goal>
    +                        </goals>
    +                    </execution>
    +                </executions>
    +            </plugin>
    +            <plugin>
    +                <artifactId>maven-assembly-plugin</artifactId>
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37898379
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/mapper/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    Can delete this as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37898773
  
    --- Diff: external/storm-solr/src/test/java/org/apache/storm/solr/topology/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    Can remove this file.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37899232
  
    --- Diff: pom.xml ---
    @@ -169,8 +169,9 @@
             <module>external/storm-redis</module>
             <module>external/storm-eventhubs</module>
             <module>external/flux</module>
    -        <module>examples/storm-starter</module>
             <module>external/storm-elasticsearch</module>
    +        <module>external/storm-solr</module>
    +		<module>examples/storm-starter</module>
    --- End diff --
    
    Whitespace?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36990730
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/AbstractSolrBolt.java ---
    @@ -0,0 +1,33 @@
    +package org.apache.storm.solr.bolt;
    --- End diff --
    
    Do we need this class since we only have SolrUpdateBolt or do you have plans on adding any other bolts extending this. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-134782918
  
    +1 LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37883234
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java ---
    @@ -0,0 +1,107 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.apache.storm.solr.config.SolrCommitStrategy;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.apache.storm.solr.mapper.SolrMapper;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/19/15.
    + */
    +public class SolrUpdateBolt extends AbstractSolrBolt {
    +    private final Logger logger = LoggerFactory.getLogger(SolrUpdateBolt.class);
    +    private final SolrMapper solrMapper;
    +    private final SolrCommitStrategy commitStgy;
    +    private List<Tuple> toCommitTuples;
    +    private final String ackFailLock = "LOCK";      //serializable lock
    +
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper) {
    +        this(solrConfig, solrMapper, null);
    +    }
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper, SolrCommitStrategy commitStgy) {
    +        super(solrConfig);
    +        this.solrMapper = solrMapper;
    +        this.commitStgy = commitStgy;
    +        logger.info("Created {} with the following configuration: " +
    +                    "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = {}]",
    +                    this.getClass().getSimpleName(), solrConfig, solrMapper, commitStgy);
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        super.prepare(stormConf, context, collector);
    +        this.toCommitTuples = new LinkedList<>();
    +    }
    +
    +    @Override
    +    public void execute(Tuple tuple) {
    +        try {
    +            SolrRequest request = solrMapper.toSolrRequest(tuple);
    +            solrClient.request(request, solrMapper.getCollection());
    +            ack(tuple);
    +        } catch (Exception e) {
    +            fail(tuple, e);
    +        }
    +    }
    +
    +    private void ack(Tuple tuple) throws SolrServerException, IOException {
    +        if (commitStgy == null) {
    +            collector.ack(tuple);
    +        } else {
    +            synchronized(ackFailLock) {
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36987568
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java ---
    @@ -0,0 +1,107 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.apache.storm.solr.config.SolrCommitStrategy;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.apache.storm.solr.mapper.SolrMapper;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/19/15.
    + */
    +public class SolrUpdateBolt extends AbstractSolrBolt {
    +    private final Logger logger = LoggerFactory.getLogger(SolrUpdateBolt.class);
    +    private final SolrMapper solrMapper;
    +    private final SolrCommitStrategy commitStgy;
    +    private List<Tuple> toCommitTuples;
    +    private final String ackFailLock = "LOCK";      //serializable lock
    +
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper) {
    +        this(solrConfig, solrMapper, null);
    +    }
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper, SolrCommitStrategy commitStgy) {
    +        super(solrConfig);
    +        this.solrMapper = solrMapper;
    +        this.commitStgy = commitStgy;
    +        logger.info("Created {} with the following configuration: " +
    +                    "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = {}]",
    +                    this.getClass().getSimpleName(), solrConfig, solrMapper, commitStgy);
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        super.prepare(stormConf, context, collector);
    +        this.toCommitTuples = new LinkedList<>();
    --- End diff --
    
    any reason to use LinkedList over ArrayList.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37883211
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/AbstractSolrBolt.java ---
    @@ -0,0 +1,33 @@
    +package org.apache.storm.solr.bolt;
    --- End diff --
    
    I deleted this class as I was able to abstract the specific implementations in the mapper, which returns the SolrRequest object


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-127483109
  
    I am planning on pushing a few more unit tests while the community does the review. I have provided a set of functional tests that use the Solr gettingstarted example. I ran the functional tests successfully with Solr running locally. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37898067
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/.gitignore ---
    @@ -0,0 +1 @@
    +# Created by .ignore support plugin (hsz.mobi)
    --- End diff --
    
    This file can probably be deleted.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/665


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37883322
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java ---
    @@ -0,0 +1,107 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrRequest;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.apache.storm.solr.config.SolrCommitStrategy;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.apache.storm.solr.mapper.SolrMapper;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/19/15.
    + */
    +public class SolrUpdateBolt extends AbstractSolrBolt {
    +    private final Logger logger = LoggerFactory.getLogger(SolrUpdateBolt.class);
    +    private final SolrMapper solrMapper;
    +    private final SolrCommitStrategy commitStgy;
    +    private List<Tuple> toCommitTuples;
    +    private final String ackFailLock = "LOCK";      //serializable lock
    +
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper) {
    +        this(solrConfig, solrMapper, null);
    +    }
    +
    +    public SolrUpdateBolt(SolrConfig solrConfig, SolrMapper solrMapper, SolrCommitStrategy commitStgy) {
    +        super(solrConfig);
    +        this.solrMapper = solrMapper;
    +        this.commitStgy = commitStgy;
    +        logger.info("Created {} with the following configuration: " +
    +                    "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = {}]",
    +                    this.getClass().getSimpleName(), solrConfig, solrMapper, commitStgy);
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
    +        super.prepare(stormConf, context, collector);
    +        this.toCommitTuples = new LinkedList<>();
    --- End diff --
    
    It now uses ArrayList with the correct size at creation, to avoid reallocation overhead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-131583675
  
    I volunteer to be sponsor for this connector


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37928702
  
    --- Diff: external/storm-solr/README.md ---
    @@ -0,0 +1,188 @@
    +# Storm Solr
    +Storm and Trident integration for Apache Solr. This package includes a bolt and a trident state that enable a Storm topology
    +stream the contents of storm tuples to index Solr collections.
    + 
    +# Index Storm tuples into a Solr collection
    +The The bolt and trident state provided use one of the supplied mappers to build a `SolrRequest` object that is 
    +responsible for making the update calls to Solr, thus updating the index of the collection specified.
    + 
    +# Usage Examples 
    +In this section we provide some simple code snippets on how to build Storm and Trident topologies to index Solr. In subsequent sections we 
    +describe in detail the two key components of the Storm Solr integration, the `SolrUpdateBolt`, and the `Mappers`, `SolrFieldsMapper`, and `SolrJsonMapper`.
    +
    +## Storm Bolt With JSON Mapper and Count Based Commit Strategy
    +
    +```java
    +    new SolrUpdateBolt(solrConfig, solrMapper, solrCommitStgy)
    +    
    +    // zkHostString for Solr 'gettingstarted' example
    +    SolrConfig solrConfig = new SolrConfig("127.0.0.1:9983");
    +    
    +    // JSON Mapper used to generate 'SolrRequest' requests to update the "gettingstarted" Solr collection with JSON content declared the tuple field with name "JSON"
    +    SolrMapper solrMapper = new SolrJsonMapper.Builder("gettingstarted", "JSON").build(); 
    +     
    +    // Acks every other five tuples. Setting to null acks every tuple
    +    SolrCommitStrategy solrCommitStgy = new CountBasedCommit(5);          
    +```
    +
    +## Trident Topology With Fields Mapper
    +```java
    +    new SolrStateFactory(solrConfig, solrMapper);
    +    
    +    // zkHostString for Solr 'gettingstarted' example
    +    SolrConfig solrConfig = new SolrConfig("127.0.0.1:9983");
    +    
    +    /* Solr Fields Mapper used to generate 'SolrRequest' requests to update the "gettingstarted" Solr collection. The Solr index is updated using the field values of the tuple fields that match static or dynamic fields declared in the schema object build using schemaBuilder */ 
    +    SolrMapper solrMapper = new SolrFieldsMapper.Builder(schemaBuilder, 'gettingstarted').build();
    +
    +    // builds the Schema object from the JSON representation of the schema as returned by the URL http://localhost:8983/solr/gettingstarted/schema/ 
    +    SchemaBuilder schemaBuilder = new RestJsonSchemaBuilder("localhost", "8983", "gettingstarted")
    +```
    +
    +## SolrUpdateBolt
    + `SolrUpdateBolt` streams tuples directly into Apache Solr. The Solr index is updated using`SolrRequest` requests. 
    + The `SolrUpdateBolt` is configurable using implementations of `SolrConfig`, `SolrMapper`, and optionally `SolrCommitStrategy`.
    +   
    + The data to stream onto Solr is extracted from the tuples using the strategy defined in the `SolrMapper` implementation.
    + 
    + The `SolrRquest` can be sent every tuple, or according to a strategy defined by `SolrCommitStrategy` implementations. 
    + If a `SolrCommitStrategy` is in place and one of the tuples in the batch fails, the batch is not committed, and all the tuples in that 
    +  batch are marked as Fail, and retried. On the other hand, if all tuples succeed, the `SolrRequest` is committed and all tuples are successfully acked.
    + 
    + `SolrConfig` is the class containing Solr configuration to be made available to Storm Solr bolts. Any configuration needed in the bolts should be put in this class.
    + 
    +
    +## SolrMapper
    +`SorlMapper` implementations define the strategy to extract information from the tuples. The public method
    +`toSolrRequest` receives a tuple or a list of tuples and returns a `SolrRequest` object that is used to update the Solr index.
    +
    +
    +### SolrJsonMapper
    +The `SolrJsonMapper` creates a Solr update request that is sent to the URL endpoint defined by Solr as the resource 
    +destination for requests in JSON format. 
    + 
    +To create a `SolrJsonMapper` the client must specify the name of the collection to update as well as the 
    +tuple field that contains the JSON object used to update the Solr index. If the tuple does not contain the field specified, 
    +a `SolrMapperException` will be thrown when the method `toSolrRequest`is called. If the field exists, its value can either 
    +be a String with the contents in JSON format, or a Java object that will be serialized to JSON
    + 
    +Code snippet illustrating how to create a `SolrJsonMapper` object to update the `gettingstarted` Solr collection with JSON content 
    +declared in the tuple field with name "JSON"
    +``` java
    +    SolrMapper solrMapper = new SolrJsonMapper.Builder("gettingstarted", "JSON").build();
    +```
    +
    +
    +### SolrFieldsMapper
    +The `SolrFieldsMapper` creates a Solr update request that is sent to the Solr URL endpoint that handles the updates of `SolrInputDocument` objects.
    +
    +To create a `SolrFieldsMapper` the client must specify the name of the collection to update as well as the `SolrSchemaBuilder`. 
    +The Solr `Schema` is used to extract information about the Solr schema fields and corresponding types. This metadata is used
    +to get the information from the tuples. Only tuple fields that match a static or dynamic Solr fields are added to the document. Tuple fields 
    +that do not match the schema are not added to the `SolrInputDocument` being prepared for indexing. A debug log message is printed for the 
    + tuple fields that do not match the schema and hence are not indexed.
    + 
    +
    +The `SolrFieldsMapper` supports multivalue fields. A multivalue tuple field must be tokenized. The default token is |. Any 
    +arbitrary token can be specified by calling the method `org.apache.storm.solr.mapper.SolrFieldsMapper.Builder.setMultiValueFieldToken`
    +that is part of the `SolrFieldsMapper.Builder` builder class. 
    +
    +Code snippet illustrating how to create a `SolrFieldsMapper` object to update the `gettingstarted` Solr collection. The multivalue 
    +field separates each value with the token % instead of the default | . To use the default token you can ommit the call to the method
    +`setMultiValueFieldToken`.
    +
    +``` java
    +    new SolrFieldsMapper.Builder(
    +            new RestJsonSchemaBuilder("localhost", "8983", `gettingstarted`), `gettingstarted`)
    +                .setMultiValueFieldToken("%").build();
    +```
    +
    +# Build And Run Bundled Examples  
    +To be able to run the examples you must first build the java code in the package `storm-solr`, 
    +and then generate an uber jar with all the dependencies.
    +
    +
    +## Build the Storm Apache Solr Integration Code
    +
    +`mvn clean install -f REPO_HOME/storm/external/storm-solr`
    + 
    +## Use the Maven Assembly Plugin to Build the Uber Jar
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37901014
  
    --- Diff: external/storm-solr/pom.xml ---
    @@ -0,0 +1,106 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<project xmlns="http://maven.apache.org/POM/4.0.0"
    +         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <artifactId>storm</artifactId>
    +        <groupId>org.apache.storm</groupId>
    +        <version>0.11.0-SNAPSHOT</version>
    +        <relativePath>../../pom.xml</relativePath>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +
    +    <artifactId>storm-solr</artifactId>
    +
    +    <developers>
    +        <developer>
    +            <id>Hugo-Louro</id>
    +            <name>Hugo Louro</name>
    +            <email>hmclouro@gmail.com</email>
    +        </developer>
    +    </developers>
    +
    +    <dependencies>
    +        <dependency>
    +            <groupId>org.apache.storm</groupId>
    +            <artifactId>storm-core</artifactId>
    +            <version>${project.version}</version>
    +            <scope>provided</scope>
    +        </dependency>
    +        <!-- Solr and its dependencies -->
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-solrj</artifactId>
    +            <version>5.2.1</version>
    +            <scope>compile</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-codec</groupId>
    +            <artifactId>commons-codec</artifactId>
    +            <version>1.3</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-httpclient</groupId>
    +            <artifactId>commons-httpclient</artifactId>
    +            <version>3.1</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>commons-io</groupId>
    +            <artifactId>commons-io</artifactId>
    +            <version>1.4</version>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-core</artifactId>
    +            <version>5.2.1</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.solr</groupId>
    +            <artifactId>solr-test-framework</artifactId>
    +            <version>5.2.1</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>com.google.guava</groupId>
    +            <artifactId>guava</artifactId>
    +            <version>17.0</version>
    +        </dependency>
    +        <!--test dependencies -->
    +        <dependency>
    +            <groupId>junit</groupId>
    +            <artifactId>junit</artifactId>
    +            <version>4.11</version>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>com.google.code.gson</groupId>
    +            <artifactId>gson</artifactId>
    +            <version>2.3.1</version>
    +        </dependency>
    +    </dependencies>
    +    <build>
    +        <plugins>
    +            <plugin>
    +                <groupId>org.apache.maven.plugins</groupId>
    +                <artifactId>maven-jar-plugin</artifactId>
    +                <version>2.5</version>
    +                <executions>
    +                    <execution>
    +                        <goals>
    +                            <goal>test-jar</goal>
    +                        </goals>
    +                    </execution>
    +                </executions>
    +            </plugin>
    +            <plugin>
    +                <artifactId>maven-assembly-plugin</artifactId>
    --- End diff --
    
    I'd prefer to see the Maven shade plugin used instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37147769
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/schema/builder/RestJsonSchemaBuilder.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.storm.solr.schema.builder;
    +
    +import com.google.gson.Gson;
    +import org.apache.storm.solr.schema.Schema;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.Scanner;
    +
    +/**
    + * Class that buils the {@link Schema} object from the JSON representation of the schema as returned by the
    + * URL of the form http://localhost:8983/solr/gettingstarted/schema/ . This particular URL returns the schema
    + * in JSON format for the gettingstarted example running locally.
    + * <p></p>
    + * Created by hlouro on 7/28/15.
    + */
    +public class RestJsonSchemaBuilder implements SchemaBuilder {
    +    private static final Logger logger = LoggerFactory.getLogger(RestJsonSchemaBuilder.class);
    +    private Schema schema;
    +
    +
    +    /** Urls with the form http://localhost:8983/solr/gettingstarted/schema/ returns the schema in JSON format */
    +    public RestJsonSchemaBuilder(String solrHost, String solrPort, String collection) throws IOException {
    +        this(new URL("http://" + solrHost + ":" + solrPort + "/solr/" + collection + "/schema/"));
    --- End diff --
    
    my bad overlooked the constructor with url this should be enough.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the pull request:

    https://github.com/apache/storm/pull/665#issuecomment-135494026
  
    Great work @hmcl. Merged into trunk. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by hmcl <gi...@git.apache.org>.
Github user hmcl commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r37883364
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/AbstractSolrBolt.java ---
    @@ -0,0 +1,33 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.topology.base.BaseRichBolt;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/17/15.
    --- End diff --
    
    Done. Also added the Apache Copyright.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: STORM-851: Storm Solr Connector

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/665#discussion_r36863654
  
    --- Diff: external/storm-solr/src/main/java/org/apache/storm/solr/bolt/AbstractSolrBolt.java ---
    @@ -0,0 +1,33 @@
    +package org.apache.storm.solr.bolt;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.task.TopologyContext;
    +import backtype.storm.topology.OutputFieldsDeclarer;
    +import backtype.storm.topology.base.BaseRichBolt;
    +import backtype.storm.tuple.Tuple;
    +import org.apache.solr.client.solrj.SolrClient;
    +import org.apache.solr.client.solrj.impl.CloudSolrClient;
    +import org.apache.storm.solr.config.SolrConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +/**
    + * Created by hlouro on 7/17/15.
    --- End diff --
    
    can you remove these comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---