You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@streams.apache.org by steveblackmon <gi...@git.apache.org> on 2014/07/31 19:29:58 UTC

[GitHub] incubator-streams pull request: Streams 138

GitHub user steveblackmon opened a pull request:

    https://github.com/apache/incubator-streams/pull/59

    Streams 138

    basic push implementation w/ small fixes to jsonschema and serializer
    see https://github.com/w2ogroup/streams-examples/tree/master/datasift-webhook
    for an example of how to deploy this provider

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

    $ git pull https://github.com/apache/incubator-streams STREAMS-138

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

    https://github.com/apache/incubator-streams/pull/59.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 #59
    
----
commit f9d484b8421640e501e84f1c056b079a1404446d
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-15T21:59:23Z

    additions to enable streams-api

commit d9e0a384cda3820d53d40d47329fd65fb09c1282
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-28T00:13:25Z

    adding push provider, reverting change to streams provider

commit 9e774a8a37f27187766eb02255f4a900ae37b711
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:22:22Z

    simplify/isolate push provider

commit bdf30f78a31e4a4a8a11e979488b62f3d914c9a1
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:22:57Z

    added missing fields
    switched a few date fields to Joda

commit b7301326a83936a528219c5b84b35a86a57d5971
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:24:14Z

    created a DatasiftObjectMapper capable of parsing either RFC3339 or the standard Datasift date string
    changed all classes in package except configurator to use this Mapper

commit 01a1a73be4926c9d518c5d1afb10f7b54db489f5
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:25:17Z

    added null check to get rid of common exception seen while stream initializing

commit eda5a3cb77f512fa4e1094a0bf74a3c35c63605b
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-15T21:59:23Z

    additions to enable streams-api

commit f272ff5347f0a00b24402960b251c143e966f34f
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-28T00:13:25Z

    adding push provider, reverting change to streams provider

commit 2e66e51a9fbef1e1fb7ae922a37d94d38dcf254a
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:22:22Z

    simplify/isolate push provider

commit fec8a37ca99df822616513727c618853a8a71fdc
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:22:57Z

    added missing fields
    switched a few date fields to Joda

commit 3af77bd1ff83e4c882eb90ea899a02661fdbd2ee
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:24:14Z

    created a DatasiftObjectMapper capable of parsing either RFC3339 or the standard Datasift date string
    changed all classes in package except configurator to use this Mapper

commit e7141a4d1a7fcaa8c99827f02257ee2987e6d826
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:25:17Z

    added null check to get rid of common exception seen while stream initializing

commit 5dec4aadb5947614fb7beb8b631c017adaea0fec
Author: Steve Blackmon <sb...@w2odigital.com>
Date:   2014-07-31T02:27:32Z

    Merge branch 'streams-api' of https://git-wip-us.apache.org/repos/asf/incubator-streams into streams-api

----


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59#discussion_r15809254
  
    --- Diff: streams-contrib/streams-provider-datasift/src/main/java/org/apache/streams/datasift/provider/DatasiftPushProvider.java ---
    @@ -0,0 +1,133 @@
    +/*
    +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.streams.datasift.provider;
    +
    +import com.datasift.client.stream.DeletedInteraction;
    +import com.datasift.client.stream.StreamEventListener;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import org.apache.streams.core.StreamsDatum;
    +import org.apache.streams.core.StreamsProvider;
    +import org.apache.streams.core.StreamsResultSet;
    +import org.apache.streams.datasift.DatasiftConfiguration;
    +import org.joda.time.DateTime;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.math.BigInteger;
    +import java.util.Queue;
    +import java.util.concurrent.locks.ReadWriteLock;
    +import java.util.concurrent.locks.ReentrantReadWriteLock;
    +
    +/**
    + * Requires Java Version 1.7!
    + * {@code DatasiftStreamProvider} is an implementation of the {@link org.apache.streams.core.StreamsProvider} interface.  The provider
    + * uses the Datasift java api to make connections. A single provider creates one connection per StreamHash in the configuration.
    + */
    +public class DatasiftPushProvider implements StreamsProvider {
    --- End diff --
    
    It seems like this class doesn't do anything at the moment.  Is that intentional?  
    
    Also, why not tie into the existing Streams web runtime for publication?


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59#discussion_r15809084
  
    --- Diff: streams-contrib/streams-provider-datasift/src/main/java/org/apache/streams/datasift/util/StreamsDatasiftMapper.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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
    + *
    + *   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.streams.datasift.util;
    +
    +import com.fasterxml.jackson.core.JsonParser;
    +import com.fasterxml.jackson.core.JsonProcessingException;
    +import com.fasterxml.jackson.databind.DeserializationContext;
    +import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
    +import com.fasterxml.jackson.databind.module.SimpleModule;
    +import org.apache.streams.data.util.RFC3339Utils;
    +import org.apache.streams.jackson.StreamsJacksonMapper;
    +import org.joda.time.DateTime;
    +import org.joda.time.format.DateTimeFormat;
    +import org.joda.time.format.DateTimeFormatter;
    +
    +import java.io.IOException;
    +
    +/**
    + * Created by sblackmon on 3/27/14.
    + */
    +public class StreamsDatasiftMapper extends StreamsJacksonMapper {
    +
    +    public static final DateTimeFormatter DATASIFT_FORMAT = DateTimeFormat.forPattern("EEE, dd MMM yyyy HH:mm:ss Z");
    --- End diff --
    
    Wouldn't the need for this be obviated if you updated RFC339Utils with this pattern?


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59#discussion_r16003460
  
    --- Diff: streams-contrib/streams-provider-datasift/src/main/java/org/apache/streams/datasift/util/StreamsDatasiftMapper.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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
    + *
    + *   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.streams.datasift.util;
    +
    +import com.fasterxml.jackson.core.JsonParser;
    +import com.fasterxml.jackson.core.JsonProcessingException;
    +import com.fasterxml.jackson.databind.DeserializationContext;
    +import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
    +import com.fasterxml.jackson.databind.module.SimpleModule;
    +import org.apache.streams.data.util.RFC3339Utils;
    +import org.apache.streams.jackson.StreamsJacksonMapper;
    +import org.joda.time.DateTime;
    +import org.joda.time.format.DateTimeFormat;
    +import org.joda.time.format.DateTimeFormatter;
    +
    +import java.io.IOException;
    +
    +/**
    + * Created by sblackmon on 3/27/14.
    + */
    +public class StreamsDatasiftMapper extends StreamsJacksonMapper {
    +
    +    public static final DateTimeFormatter DATASIFT_FORMAT = DateTimeFormat.forPattern("EEE, dd MMM yyyy HH:mm:ss Z");
    --- End diff --
    
    opening STREAMS-143 to address this issue


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59#discussion_r15817312
  
    --- Diff: streams-contrib/streams-provider-datasift/src/main/java/org/apache/streams/datasift/util/StreamsDatasiftMapper.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * 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
    + *
    + *   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.streams.datasift.util;
    +
    +import com.fasterxml.jackson.core.JsonParser;
    +import com.fasterxml.jackson.core.JsonProcessingException;
    +import com.fasterxml.jackson.databind.DeserializationContext;
    +import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
    +import com.fasterxml.jackson.databind.module.SimpleModule;
    +import org.apache.streams.data.util.RFC3339Utils;
    +import org.apache.streams.jackson.StreamsJacksonMapper;
    +import org.joda.time.DateTime;
    +import org.joda.time.format.DateTimeFormat;
    +import org.joda.time.format.DateTimeFormatter;
    +
    +import java.io.IOException;
    +
    +/**
    + * Created by sblackmon on 3/27/14.
    + */
    +public class StreamsDatasiftMapper extends StreamsJacksonMapper {
    +
    +    public static final DateTimeFormatter DATASIFT_FORMAT = DateTimeFormat.forPattern("EEE, dd MMM yyyy HH:mm:ss Z");
    --- End diff --
    
    Perhaps - it would be great if there were one ObjectMapper implementation that would be compatible with all existing and future modules and any datetime format anywhere on the web.  I'm all for trying to build one, but that goes beyond the scope of this story.  In any case I think it's appropriate for modules that require tweaks to StreamsJacksonMapper to extend it internally, rather than updating streams-pojo.  
    However I'd like to see something in streams-pojo that could determine the appropriate Mapper for any document, even if the Mapper was in a contrib module, and perhaps run all serialization/deserialization operations through a utility class.


---
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] incubator-streams pull request: Streams 138

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

    https://github.com/apache/incubator-streams/pull/59#discussion_r15817614
  
    --- Diff: streams-contrib/streams-provider-datasift/src/main/java/org/apache/streams/datasift/provider/DatasiftPushProvider.java ---
    @@ -0,0 +1,133 @@
    +/*
    +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.streams.datasift.provider;
    +
    +import com.datasift.client.stream.DeletedInteraction;
    +import com.datasift.client.stream.StreamEventListener;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import org.apache.streams.core.StreamsDatum;
    +import org.apache.streams.core.StreamsProvider;
    +import org.apache.streams.core.StreamsResultSet;
    +import org.apache.streams.datasift.DatasiftConfiguration;
    +import org.joda.time.DateTime;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.math.BigInteger;
    +import java.util.Queue;
    +import java.util.concurrent.locks.ReadWriteLock;
    +import java.util.concurrent.locks.ReentrantReadWriteLock;
    +
    +/**
    + * Requires Java Version 1.7!
    + * {@code DatasiftStreamProvider} is an implementation of the {@link org.apache.streams.core.StreamsProvider} interface.  The provider
    + * uses the Datasift java api to make connections. A single provider creates one connection per StreamHash in the configuration.
    + */
    +public class DatasiftPushProvider implements StreamsProvider {
    --- End diff --
    
    This class is meant to be extended / wrapped in a manner similar to this:
    https://github.com/w2ogroup/streams-examples/tree/master/datasift-webhook
    
    If there is a generic method already in streams to collect json documents from a ServerSocket and route them into downstream processors and persisters, we should explore that and deprecate this approach.  It was not obvious to me how to make that happen within the existing code base.


---
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.
---