You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streams.apache.org by sb...@apache.org on 2014/09/05 21:56:15 UTC

git commit: new examples, new ES processors to make them possible

Repository: incubator-streams
Updated Branches:
  refs/heads/pp ebfc4b4e7 -> d3ca0e7d0


new examples, new ES processors to make them possible


Project: http://git-wip-us.apache.org/repos/asf/incubator-streams/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-streams/commit/d3ca0e7d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-streams/tree/d3ca0e7d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-streams/diff/d3ca0e7d

Branch: refs/heads/pp
Commit: d3ca0e7d0f6c0ea4bd09fbd8f62f45d45fa068e0
Parents: ebfc4b4
Author: sblackmon <sb...@apache.org>
Authored: Fri Sep 5 14:55:48 2014 -0500
Committer: sblackmon <sb...@apache.org>
Committed: Fri Sep 5 14:55:48 2014 -0500

----------------------------------------------------------------------
 .../ElasticsearchPersistDeleter.java            | 31 ++++---
 .../ElasticsearchPersistWriter.java             |  2 +-
 .../processor/DatumFromMetadataProcessor.java   | 92 ++++++++++++++++++++
 .../processor/MetadataAsDocumentProcessor.java  | 69 +++++++++++++++
 .../peoplepattern/AccountTypeProcessor.java     | 57 +++++++-----
 .../peoplepattern/DemographicsProcessor.java    | 56 +++++++-----
 .../streams/peoplepattern/AccountType.json      |  1 +
 7 files changed, 248 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistDeleter.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistDeleter.java b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistDeleter.java
index fece72e..7d74959 100644
--- a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistDeleter.java
+++ b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistDeleter.java
@@ -28,6 +28,8 @@ import org.elasticsearch.action.update.UpdateRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Map;
+
 public class ElasticsearchPersistDeleter extends ElasticsearchPersistWriter implements StreamsPersistWriter {
 
     private final static Logger LOGGER = LoggerFactory.getLogger(ElasticsearchPersistDeleter.class);
@@ -44,21 +46,22 @@ public class ElasticsearchPersistDeleter extends ElasticsearchPersistWriter impl
     public void write(StreamsDatum streamsDatum) {
 
         Preconditions.checkNotNull(streamsDatum);
-        Preconditions.checkNotNull(streamsDatum.getDocument());
         Preconditions.checkNotNull(streamsDatum.getMetadata());
-        Preconditions.checkNotNull(streamsDatum.getMetadata().get("id"));
-
-        String index;
-        String type;
-        String id;
-
-        index = Optional.fromNullable(
-                (String) streamsDatum.getMetadata().get("index"))
-                .or(config.getIndex());
-        type = Optional.fromNullable(
-                (String) streamsDatum.getMetadata().get("type"))
-                .or(config.getType());
-        id = (String) streamsDatum.getMetadata().get("id");
+
+        LOGGER.debug("Delete Metadata: {}", streamsDatum.getMetadata());
+
+        String index = (String) streamsDatum.getMetadata().get("index");
+        String type = (String) streamsDatum.getMetadata().get("type");
+        String id = setId(streamsDatum);
+
+        if(index == null || (config.getForceUseConfig() != null && config.getForceUseConfig())) {
+            index = config.getIndex();
+        }
+        if(type == null || (config.getForceUseConfig() != null && config.getForceUseConfig())) {
+            type = config.getType();
+        }
+
+        LOGGER.debug("Attempt Delete: ({},{},{})", index, type, id);
 
         delete(index, type, id);
 

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistWriter.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistWriter.java b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistWriter.java
index 664dd24..8300c62 100644
--- a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistWriter.java
+++ b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/ElasticsearchPersistWriter.java
@@ -164,7 +164,7 @@ public class ElasticsearchPersistWriter implements StreamsPersistWriter, DatumSt
         }
     }
 
-    private String setId(StreamsDatum streamsDatum) {
+    protected String setId(StreamsDatum streamsDatum) {
         String id = Optional.fromNullable(
                 (String) streamsDatum.getMetadata().get("id"))
                 .orNull();

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/DatumFromMetadataProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/DatumFromMetadataProcessor.java b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/DatumFromMetadataProcessor.java
new file mode 100644
index 0000000..5e58190
--- /dev/null
+++ b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/DatumFromMetadataProcessor.java
@@ -0,0 +1,92 @@
+package org.apache.streams.elasticsearch.processor;
+
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProcessor;
+import org.apache.streams.elasticsearch.ElasticsearchClientManager;
+import org.apache.streams.elasticsearch.ElasticsearchConfigurator;
+import org.apache.streams.elasticsearch.ElasticsearchReaderConfiguration;
+import org.elasticsearch.action.get.GetRequest;
+import org.elasticsearch.action.get.GetRequestBuilder;
+import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.index.engine.Engine;
+import org.joda.time.DateTime;
+
+import java.util.List;
+
+/**
+ * Created by sblackmon on 9/4/14.
+ */
+public class DatumFromMetadataProcessor implements StreamsProcessor {
+
+    public final static String STREAMS_ID = "DatumFromMetadataProcessor";
+
+    private ElasticsearchClientManager elasticsearchClientManager;
+    private ElasticsearchReaderConfiguration config;
+
+    public DatumFromMetadataProcessor() {
+        Config config = StreamsConfigurator.config.getConfig("elasticsearch");
+        this.config = ElasticsearchConfigurator.detectReaderConfiguration(config);
+    }
+
+    public DatumFromMetadataProcessor(Config config) {
+        this.config = ElasticsearchConfigurator.detectReaderConfiguration(config);
+    }
+
+    public DatumFromMetadataProcessor(ElasticsearchReaderConfiguration config) {
+        this.config = config;
+    }
+
+    @Override
+    public List<StreamsDatum> process(StreamsDatum entry) {
+        List<StreamsDatum> result = Lists.newArrayList();
+
+        if(entry == null || entry.getMetadata() == null)
+            return result;
+
+        String index = (String) entry.getMetadata().get("index");
+        String type = (String) entry.getMetadata().get("type");
+        String id = (String) entry.getMetadata().get("id");
+
+        if( index == null ) {
+            index = this.config.getIndexes().get(0);
+        }
+        if( type == null ) {
+            type = this.config.getTypes().get(0);
+        }
+        if( id == null ) {
+            id = entry.getId();
+        }
+
+        GetRequestBuilder getRequestBuilder = elasticsearchClientManager.getClient().prepareGet(index, type, id);
+        getRequestBuilder.setFields("*", "_timestamp");
+        getRequestBuilder.setFetchSource(true);
+        GetResponse getResponse = getRequestBuilder.get();
+
+        if( getResponse == null || getResponse.isExists() == false || getResponse.isSourceEmpty() == true )
+            return result;
+
+        entry.setDocument(getResponse.getSource());
+        if( getResponse.getField("_timestamp") != null) {
+            DateTime timestamp = new DateTime(((Long) getResponse.getField("_timestamp").getValue()).longValue());
+            entry.setTimestamp(timestamp);
+        }
+
+        result.add(entry);
+
+        return result;
+    }
+
+    @Override
+    public void prepare(Object configurationObject) {
+        this.elasticsearchClientManager = new ElasticsearchClientManager(config);
+
+    }
+
+    @Override
+    public void cleanUp() {
+        this.elasticsearchClientManager.getClient().close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/MetadataAsDocumentProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/MetadataAsDocumentProcessor.java b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/MetadataAsDocumentProcessor.java
new file mode 100644
index 0000000..7a67760
--- /dev/null
+++ b/streams-contrib/streams-persist-elasticsearch/src/main/java/org/apache/streams/elasticsearch/processor/MetadataAsDocumentProcessor.java
@@ -0,0 +1,69 @@
+package org.apache.streams.elasticsearch.processor;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.datatype.jsonorg.JsonOrgModule;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProcessor;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Created by sblackmon on 9/5/14.
+ */
+public class MetadataAsDocumentProcessor implements StreamsProcessor {
+
+    public final static String STREAMS_ID = "DocumentToMetadataProcessor";
+
+    private ObjectMapper mapper;
+
+    @Override
+    public void prepare(Object configurationObject) {
+        mapper = StreamsJacksonMapper.getInstance();
+        mapper.registerModule(new JsonOrgModule());
+    }
+
+    @Override
+    public void cleanUp() {
+
+    }
+
+    @Override
+    public List<StreamsDatum> process(StreamsDatum entry) {
+        List<StreamsDatum> result = Lists.newArrayList();
+        ObjectNode metadataObjectNode;
+        try {
+            metadataObjectNode = mapper.readValue((String) entry.getDocument(), ObjectNode.class);
+        } catch (IOException e) {
+            return result;
+        }
+
+        Map<String, Object> metadata = asMap(metadataObjectNode);
+        entry.setMetadata(metadata);
+        result.add(entry);
+        return result;
+    }
+
+    public Map<String, Object> asMap(JsonNode node) {
+
+        Iterator<Map.Entry<String, JsonNode>> iterator = node.fields();
+        Map<String, Object> ret = Maps.newHashMap();
+
+        Map.Entry<String, JsonNode> entry;
+
+        while (iterator.hasNext()) {
+            entry = iterator.next();
+            if( entry.getValue().asText() != null )
+                ret.put(entry.getKey(), entry.getValue().asText());
+        }
+
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/AccountTypeProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/AccountTypeProcessor.java b/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/AccountTypeProcessor.java
index e412639..cf5e853 100644
--- a/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/AccountTypeProcessor.java
+++ b/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/AccountTypeProcessor.java
@@ -57,7 +57,7 @@ public class AccountTypeProcessor implements StreamsProcessor {
 
     private final static Logger LOGGER = LoggerFactory.getLogger(AccountTypeProcessor.class);
 
-    private ObjectMapper mapper = StreamsJacksonMapper.getInstance();
+    private ObjectMapper mapper;
 
     private PeoplePatternConfiguration peoplePatternConfiguration = null;
 
@@ -115,34 +115,17 @@ public class AccountTypeProcessor implements StreamsProcessor {
         CloseableHttpClient httpclient = HttpClients.createDefault();
 
         CloseableHttpResponse response = null;
+
+        String entityString = null;
         try {
-            //response = httpclient.execute(httpget, context);
             response = httpclient.execute(httpget);
             HttpEntity entity = response.getEntity();
             // TODO: handle rate-limiting
             if (response.getStatusLine().getStatusCode() == 200 && entity != null) {
-                String entityString = EntityUtils.toString(entity);
-
-                LOGGER.debug(entityString);
-
-                AccountType accountType = mapper.readValue(entityString, AccountType.class);
-
-                Map<String, Object> extensions = ActivityUtil.ensureExtensions(mapper.convertValue(actor, ObjectNode.class));
-
-                extensions.put(EXTENSION, accountType);
-
-                actor.setAdditionalProperty(ActivityUtil.EXTENSION_PROPERTY, extensions);
-
-                LOGGER.debug("Actor: {}", actor);
-
-                activity.setActor(actor);
-
-                entry.setDocument(activity);
-
-                result.add(entry);
+                entityString = EntityUtils.toString(entity);
             }
         } catch (IOException e) {
-            LOGGER.error("IO error {} - {}", uri.toString(), response);
+            LOGGER.error("IO error:\n{}\n{}\n{}", uri.toString(), response, e.getMessage());
             return result;
         } finally {
             try {
@@ -153,12 +136,41 @@ public class AccountTypeProcessor implements StreamsProcessor {
             } catch (IOException e) {}
         }
 
+        if( entityString == null )
+            return result;
+
+        LOGGER.debug(entityString);
+
+        AccountType accountType;
+        try {
+            accountType = mapper.readValue(entityString, AccountType.class);
+        } catch (IOException e) {
+            LOGGER.error("IO error:\n{}\n{}\n{}", uri.toString(), response, e.getMessage());
+            return result;
+        }
+
+        Map<String, Object> extensions = ActivityUtil.ensureExtensions(mapper.convertValue(actor, ObjectNode.class));
+
+        extensions.put(EXTENSION, accountType);
+
+        actor.setAdditionalProperty(ActivityUtil.EXTENSION_PROPERTY, extensions);
+
+        LOGGER.debug("Actor: {}", actor);
+
+        activity.setActor(actor);
+
+        entry.setDocument(activity);
+
+        result.add(entry);
+
+
         return result;
 
     }
 
     @Override
     public void prepare(Object configurationObject) {
+        mapper = StreamsJacksonMapper.getInstance();
         // TODO: one client object
         StringBuilder stringBuilder = new StringBuilder();
         stringBuilder.append(peoplePatternConfiguration.getUsername());
@@ -166,7 +178,6 @@ public class AccountTypeProcessor implements StreamsProcessor {
         stringBuilder.append(peoplePatternConfiguration.getPassword());
         String string = stringBuilder.toString();
         authHeader = Base64.encodeBase64String(string.getBytes());
-
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/DemographicsProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/DemographicsProcessor.java b/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/DemographicsProcessor.java
index bd7a1b2..b87ad9a 100644
--- a/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/DemographicsProcessor.java
+++ b/streams-contrib/streams-processor-peoplepattern/src/main/java/org/apache/streams/peoplepattern/DemographicsProcessor.java
@@ -51,13 +51,13 @@ import java.util.Map;
  */
 public class DemographicsProcessor implements StreamsProcessor {
 
-    private final static String STREAMS_ID = "DemographicsProcessor";
+    public final static String STREAMS_ID = "DemographicsProcessor";
 
     private final static String EXTENSION = "demographics";
 
     private final static Logger LOGGER = LoggerFactory.getLogger(DemographicsProcessor.class);
 
-    private ObjectMapper mapper = StreamsJacksonMapper.getInstance();
+    private ObjectMapper mapper;
 
     private PeoplePatternConfiguration peoplePatternConfiguration = null;
 
@@ -110,33 +110,17 @@ public class DemographicsProcessor implements StreamsProcessor {
         CloseableHttpClient httpclient = HttpClients.createDefault();
 
         CloseableHttpResponse response = null;
+
+        String entityString = null;
         try {
             response = httpclient.execute(httpget);
             HttpEntity entity = response.getEntity();
             // TODO: handle rate-limiting
             if (response.getStatusLine().getStatusCode() == 200 && entity != null) {
-                String entityString = EntityUtils.toString(entity);
-
-                LOGGER.debug(entityString);
-
-                Demographics demographics = mapper.readValue(entityString, Demographics.class);
-
-                Map<String, Object> extensions = ActivityUtil.ensureExtensions(mapper.convertValue(actor, ObjectNode.class));
-
-                extensions.put(EXTENSION, demographics);
-
-                actor.setAdditionalProperty(ActivityUtil.EXTENSION_PROPERTY, extensions);
-
-                LOGGER.debug("Actor: {}", actor);
-
-                activity.setActor(actor);
-
-                entry.setDocument(activity);
-
-                result.add(entry);
+                entityString = EntityUtils.toString(entity);
             }
         } catch (IOException e) {
-            LOGGER.error("IO error {} - {}", uri.toString(), response);
+            LOGGER.error("IO error:\n{}\n{}\n{}", uri.toString(), response, e.getMessage());
             return result;
         } finally {
             try {
@@ -147,12 +131,40 @@ public class DemographicsProcessor implements StreamsProcessor {
             } catch (IOException e) {}
         }
 
+        if( entityString == null )
+            return result;
+
+        LOGGER.debug(entityString);
+
+        Demographics demographics = null;
+        try {
+            demographics = mapper.readValue(entityString, Demographics.class);
+        } catch (IOException e) {
+            LOGGER.error("IO error:\n{}\n{}\n{}", uri.toString(), response, e.getMessage());
+            return result;
+        }
+
+        Map<String, Object> extensions = ActivityUtil.ensureExtensions(mapper.convertValue(actor, ObjectNode.class));
+
+        extensions.put(EXTENSION, demographics);
+
+        actor.setAdditionalProperty(ActivityUtil.EXTENSION_PROPERTY, extensions);
+
+        LOGGER.debug("Actor: {}", actor);
+
+        activity.setActor(actor);
+
+        entry.setDocument(activity);
+
+        result.add(entry);
+
         return result;
 
     }
 
     @Override
     public void prepare(Object configurationObject) {
+        mapper = StreamsJacksonMapper.getInstance();
         // TODO: one client object
         StringBuilder stringBuilder = new StringBuilder();
         stringBuilder.append(peoplePatternConfiguration.getUsername());

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d3ca0e7d/streams-contrib/streams-processor-peoplepattern/src/main/jsonschema/org/apache/streams/peoplepattern/AccountType.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-peoplepattern/src/main/jsonschema/org/apache/streams/peoplepattern/AccountType.json b/streams-contrib/streams-processor-peoplepattern/src/main/jsonschema/org/apache/streams/peoplepattern/AccountType.json
index 61ff110..5656b44 100644
--- a/streams-contrib/streams-processor-peoplepattern/src/main/jsonschema/org/apache/streams/peoplepattern/AccountType.json
+++ b/streams-contrib/streams-processor-peoplepattern/src/main/jsonschema/org/apache/streams/peoplepattern/AccountType.json
@@ -12,6 +12,7 @@
                 "organization",
                 "entertainment",
                 "adult",
+                "spam",
                 "no-prediction"
             ],
             "default": "no-prediction"