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/08/12 00:53:01 UTC

[01/15] git commit: tested UserInformationProvider it's running but leaving a lot of fields empty

Repository: incubator-streams
Updated Branches:
  refs/heads/master d309a5987 -> d475b913d


tested UserInformationProvider
it's running but leaving a lot of fields empty


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

Branch: refs/heads/master
Commit: 3f6a015ba605a680aeafe513c0b5cc6c26fb2c40
Parents: 49d00d7
Author: sblackmon <sb...@w2odigital.com>
Authored: Thu Jul 3 20:56:43 2014 -0700
Committer: sblackmon <sb...@w2odigital.com>
Committed: Mon Jul 21 10:30:44 2014 -0500

----------------------------------------------------------------------
 .../FacebookUserInformationProvider.java        | 116 +++++--------------
 1 file changed, 27 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/3f6a015b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
index b89a925..a167947 100644
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
@@ -64,8 +64,7 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
 
     public void setConfig(FacebookUserInformationConfiguration config)   { this.facebookUserInformationConfiguration = config; }
 
-    protected Iterator<Long[]> idsBatches;
-    protected Iterator<String[]> screenNameBatches;
+    protected Iterator<String[]> idsBatches;
 
     protected ExecutorService executor;
 
@@ -121,32 +120,9 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
         running.set(true);
     }
 
-    private void loadBatch(String[] ids) {
-        Facebook client = getFacebookClient();
-        int keepTrying = 0;
-
-        // keep trying to load, give it 5 attempts.
-        //while (keepTrying < 10)
-        while (keepTrying < 1)
-        {
-            try {
-                for( User user : client.getUsers(ids)) {
-                    String json = DataObjectFactory.getRawJSON(user);
-
-                    providerQueue.offer(new StreamsDatum(json));
-//
-                };
-            } catch (FacebookException e) {
-                e.printStackTrace();
-                return;
-            }
-
-        }
-    }
-
     public StreamsResultSet readCurrent() {
 
-//        Preconditions.checkArgument(idsBatches.hasNext() || screenNameBatches.hasNext());
+        Preconditions.checkArgument(idsBatches.hasNext());
 
         LOGGER.info("readCurrent");
 
@@ -154,48 +130,34 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
 
         try {
             User me = client.users().getMe();
-            ResponseList<Friend> friendResponseList = client.friends().getFriends(me.getId());
-            Paging<Friend> friendPaging;
-            do {
+            String json = mapper.writeValueAsString(me);
+            providerQueue.add(
+                new StreamsDatum(json, DateTime.now())
+            );
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
 
-                for( Friend friend : friendResponseList ) {
+        while( idsBatches.hasNext() ) {
+            try {
+                List<User> userList = client.users().getUsers(idsBatches.next());
+                for (User user : userList) {
 
-                    String json;
                     try {
-                        json = mapper.writeValueAsString(friend);
+                        String json = mapper.writeValueAsString(user);
                         providerQueue.add(
-                                new StreamsDatum(json)
+                            new StreamsDatum(json, DateTime.now())
                         );
                     } catch (JsonProcessingException e) {
-//                        e.printStackTrace();
+                        //                        e.printStackTrace();
                     }
                 }
-                friendPaging = friendResponseList.getPaging();
-                friendResponseList = client.fetchNext(friendPaging);
-            } while( friendPaging != null &&
-                    friendResponseList != null );
-
-//                    // Getting Next page
-//                for( Post post : feed ) {
-//                    String json;
-//                    try {
-//                        json = mapper.writeValueAsString(post);
-//                        providerQueue.add(
-//                                new StreamsDatum(json)
-//                        );
-//                    } catch (JsonProcessingException e) {
-//                        e.printStackTrace();
-//                    }
-//                }
-
-//                paging = feed.getPaging();
-//                feed = client.fetchPrevious(paging);
-//            } while( paging != null &&
-//                    feed != null );
-
 
-        } catch (FacebookException e) {
-            e.printStackTrace();
+            } catch (FacebookException e) {
+                e.printStackTrace();
+            }
         }
 
         LOGGER.info("Finished.  Cleaning up...");
@@ -260,52 +222,28 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
         Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getUserAccessToken());
         Preconditions.checkNotNull(facebookUserInformationConfiguration.getInfo());
 
-        List<String> screenNames = new ArrayList<String>();
-        List<String[]> screenNameBatches = new ArrayList<String[]>();
-
-        List<Long> ids = new ArrayList<Long>();
-        List<Long[]> idsBatches = new ArrayList<Long[]>();
+        List<String> ids = new ArrayList<String>();
+        List<String[]> idsBatches = new ArrayList<String[]>();
 
         for(String s : facebookUserInformationConfiguration.getInfo()) {
             if(s != null)
             {
-                String potentialScreenName = s.replaceAll("@", "").trim().toLowerCase();
-
-                // See if it is a long, if it is, add it to the user iD list, if it is not, add it to the
-                // screen name list
-                try {
-                    ids.add(Long.parseLong(potentialScreenName));
-                } catch (Exception e) {
-                    screenNames.add(potentialScreenName);
-                }
-
-                // Twitter allows for batches up to 100 per request, but you cannot mix types
+                ids.add(s);
 
                 if(ids.size() >= 100) {
                     // add the batch
-                    idsBatches.add(ids.toArray(new Long[ids.size()]));
+                    idsBatches.add(ids.toArray(new String[ids.size()]));
                     // reset the Ids
-                    ids = new ArrayList<Long>();
+                    ids = new ArrayList<String>();
                 }
 
-                if(screenNames.size() >= 100) {
-                    // add the batch
-                    screenNameBatches.add(screenNames.toArray(new String[ids.size()]));
-                    // reset the Ids
-                    screenNames = new ArrayList<String>();
-                }
             }
         }
 
-
         if(ids.size() > 0)
-            idsBatches.add(ids.toArray(new Long[ids.size()]));
-
-        if(screenNames.size() > 0)
-            screenNameBatches.add(screenNames.toArray(new String[ids.size()]));
+            idsBatches.add(ids.toArray(new String[ids.size()]));
 
         this.idsBatches = idsBatches.iterator();
-        this.screenNameBatches = screenNameBatches.iterator();
     }
 
     protected Facebook getFacebookClient()


[04/15] git commit: added support for FB user stream and user profile collection

Posted by sb...@apache.org.
added support for FB user stream and user profile collection


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

Branch: refs/heads/master
Commit: 49d00d7c3a65f8ef08ee9fa84f99f6290a8e796e
Parents: 7c5e290
Author: sblackmon <sb...@w2odigital.com>
Authored: Thu Jul 3 17:42:19 2014 -0700
Committer: sblackmon <sb...@w2odigital.com>
Committed: Mon Jul 21 10:30:44 2014 -0500

----------------------------------------------------------------------
 .../streams-provider-facebook/pom.xml           |  20 ++
 .../api/FacebookPostActivitySerializer.java     |  74 ++---
 .../processor/FacebookTypeConverter.java        | 194 +++++++++++
 .../FacebookUserInformationProvider.java        | 331 +++++++++++++++++++
 .../provider/FacebookUserstreamProvider.java    | 289 ++++++++++++++++
 .../com/facebook/FacebookConfiguration.json     |  49 +++
 .../FacebookUserInformationConfiguration.json   |  18 +
 .../FacebookUserstreamConfiguration.json        |  15 +
 .../jsonschema/com/facebook/graph/Post.json     |  21 +-
 .../test/FacebookActivitySerDeTest.java         |  80 +++++
 .../FacebookPostActivitySerializerTest.java     | 215 ------------
 .../facebook/test/FacebookPostSerDeTest.java    |   6 +-
 .../src/test/resources/Facebook.json            | 250 ++++++++++++++
 .../org/apache/streams/data/Facebook.json       | 251 --------------
 14 files changed, 1285 insertions(+), 528 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/pom.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/pom.xml b/streams-contrib/streams-provider-facebook/pom.xml
index 0d54255..34d53a9 100644
--- a/streams-contrib/streams-provider-facebook/pom.xml
+++ b/streams-contrib/streams-provider-facebook/pom.xml
@@ -11,6 +11,17 @@
     <artifactId>streams-provider-facebook</artifactId>
 
     <dependencies>
+
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streams</groupId>
+            <artifactId>streams-config</artifactId>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.streams</groupId>
             <artifactId>streams-pojo</artifactId>
@@ -42,6 +53,12 @@
             <artifactId>guava</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.facebook4j</groupId>
+            <artifactId>facebook4j-core</artifactId>
+            <version>2.1.0</version>
+        </dependency>
+
     </dependencies>
 
     <build>
@@ -84,6 +101,9 @@
                     <addCompileSourceRoot>true</addCompileSourceRoot>
                     <generateBuilders>true</generateBuilders>
                     <sourcePaths>
+                        <sourcePath>src/main/jsonschema/com/facebook/FacebookConfiguration.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json</sourcePath>
                         <sourcePath>src/main/jsonschema/com/facebook/graph/Post.json</sourcePath>
                     </sourcePaths>
                     <outputDirectory>target/generated-sources/jsonschema2pojo</outputDirectory>

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
index 2d53d48..71bc5c9 100644
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
@@ -29,12 +29,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializerProvider;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 import com.fasterxml.jackson.datatype.joda.JodaModule;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.streams.data.ActivitySerializer;
 import org.apache.streams.exceptions.ActivitySerializerException;
+import org.apache.streams.facebook.Post;
 import org.apache.streams.pojo.json.*;
+import org.apache.streams.jackson.StreamsJacksonMapper;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
@@ -54,7 +57,7 @@ import static org.apache.streams.data.util.JsonUtil.jsonToJsonNode;
  * Serializes activity posts
  *   sblackmon: This class needs a rewrite
  */
-public class FacebookPostActivitySerializer implements ActivitySerializer<String> {
+public class FacebookPostActivitySerializer implements ActivitySerializer<org.apache.streams.facebook.Post> {
 
     public static final DateTimeFormatter FACEBOOK_FORMAT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ssZ");
     public static final DateTimeFormatter ACTIVITY_FORMAT = ISODateTimeFormat.basicDateTime();
@@ -63,25 +66,7 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
 
     public static ObjectMapper mapper;
     static {
-        mapper = new ObjectMapper();
-        mapper.registerModule(new JodaModule());
-        mapper.disable(com.fasterxml.jackson.databind.SerializationFeature.WRITE_DATES_AS_TIMESTAMPS);
-        mapper.registerModule(new SimpleModule() {
-            {
-                addSerializer(DateTime.class, new StdSerializer<DateTime>(DateTime.class) {
-                    @Override
-                    public void serialize(DateTime value, JsonGenerator jgen, SerializerProvider provider) throws IOException, JsonGenerationException {
-                        jgen.writeString(ACTIVITY_FORMAT.print(value));
-                    }
-                });
-                addDeserializer(DateTime.class, new StdDeserializer<DateTime>(DateTime.class) {
-                    @Override
-                    public DateTime deserialize(JsonParser jpar, DeserializationContext context) throws IOException, JsonProcessingException {
-                        return FACEBOOK_FORMAT.parseDateTime(jpar.getValueAsString());
-                    }
-                });
-            }
-        });
+        mapper = StreamsJacksonMapper.getInstance();
     }
 
     @Override
@@ -90,26 +75,26 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
     }
 
     @Override
-    public String serialize(Activity deserialized) {
-        throw new UnsupportedOperationException();
+    public Post serialize(Activity deserialized) throws ActivitySerializerException {
+        throw new NotImplementedException("Not currently supported by this deserializer");
     }
 
     @Override
-    public Activity deserialize(String serialized) throws ActivitySerializerException {
-        //Deserialize the JSON string into a Jackson JsonNode
-        JsonNode node = jsonToJsonNode(serialized);
-        Map.Entry<String, JsonNode> field = getObjectType(node);
+    public Activity deserialize(Post post) throws ActivitySerializerException {
         Activity activity = new Activity();
+        activity.setPublished(post.getCreatedTime());
+        activity.setUpdated(post.getUpdatedTime());
+        addActor(activity, mapper.convertValue(post.getFrom(), ObjectNode.class));
         setProvider(activity);
-        setObjectType(field, activity);
-        parseObject(activity, field.getValue());
+        setObjectType(post.getType(), activity);
+        parseObject(activity, mapper.convertValue(post, ObjectNode.class));
         fixObjectId(activity);
         fixContentFromSummary(activity);
         return activity;
     }
 
     @Override
-    public java.util.List<Activity> deserializeAll(List<String> serializedList) {
+    public List<Activity> deserializeAll(List<Post> serializedList) {
         throw new NotImplementedException("Not currently supported by this deserializer");
     }
 
@@ -129,10 +114,10 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
         }
     }
 
-    private void setObjectType(Map.Entry<String, JsonNode> field, Activity activity) {
+    private void setObjectType(String type, Activity activity) {
         ActivityObject object = new ActivityObject();
         activity.setObject(object);
-        object.setObjectType(field.getKey());
+        object.setObjectType(type);
     }
 
     private void setProvider(Activity activity) {
@@ -142,12 +127,12 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
         activity.setProvider(provider);
     }
 
-    private Map.Entry<String, JsonNode> getObjectType(JsonNode node) {
+    private String getObjectType(JsonNode node) {
         Iterator<Map.Entry<String, JsonNode>> fields = node.fields();
         ensureMoreFields(fields);
         Map.Entry<String, JsonNode> field = fields.next();
-        ensureNoMoreFields(fields);
-        return field;
+        //ensureNoMoreFields(fields);
+        return node.asText();
     }
 
     private void parseObject(Activity activity, JsonNode jsonNode) throws ActivitySerializerException {
@@ -160,9 +145,7 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
     }
 
     private void mapField(Activity activity, String name, JsonNode value) throws ActivitySerializerException {
-        if ("created_time".equals(name)) {
-            activity.setPublished(parseDate(value));
-        } else if("application".equals(name)) {
+        if("application".equals(name)) {
             addGenerator(activity, value);
         } else if ("caption".equals(name)) {
             addSummary(activity, value);
@@ -198,16 +181,6 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
             addObjectLink(activity, value);
         } else if ("story".equals(name)) {
             addTitle(activity, value);
-        }  else if ("updated_time".equals(name)) {
-            addObjectUpdated(activity, value);
-        }
-    }
-
-    private void addObjectUpdated(Activity activity, JsonNode value) {
-        try {
-            activity.getObject().setUpdated(parseDate(value));
-        } catch( ActivitySerializerException e ) {
-            // losing this field is ok
         }
     }
 
@@ -324,11 +297,4 @@ public class FacebookPostActivitySerializer implements ActivitySerializer<String
         }
     }
 
-    private static DateTime parseDate(JsonNode value) throws ActivitySerializerException {
-        try {
-            return FACEBOOK_FORMAT.parseDateTime(value.asText());
-        } catch (Exception e) {
-            throw new ActivitySerializerException("Unable to parse date " + value.asText());
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
new file mode 100644
index 0000000..6ddb673
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
@@ -0,0 +1,194 @@
+/*
+ * 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 com.facebook.processor;
+
+import com.facebook.api.FacebookPostActivitySerializer;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.Lists;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProcessor;
+import org.apache.streams.exceptions.ActivitySerializerException;
+import org.apache.streams.facebook.Post;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.pojo.json.Activity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * Created by sblackmon on 12/10/13.
+ */
+public class FacebookTypeConverter implements StreamsProcessor {
+
+    public final static String STREAMS_ID = "TwitterTypeConverter";
+
+    private final static Logger LOGGER = LoggerFactory.getLogger(FacebookTypeConverter.class);
+
+    private ObjectMapper mapper;
+
+    private Queue<StreamsDatum> inQueue;
+    private Queue<StreamsDatum> outQueue;
+
+    private Class inClass;
+    private Class outClass;
+
+    private FacebookPostActivitySerializer facebookPostActivitySerializer;
+
+    private int count = 0;
+
+    public final static String TERMINATE = new String("TERMINATE");
+
+    public FacebookTypeConverter(Class inClass, Class outClass) {
+        this.inClass = inClass;
+        this.outClass = outClass;
+    }
+
+    public Queue<StreamsDatum> getProcessorOutputQueue() {
+        return outQueue;
+    }
+
+    public void setProcessorInputQueue(Queue<StreamsDatum> inputQueue) {
+        inQueue = inputQueue;
+    }
+
+    public Object convert(ObjectNode event, Class inClass, Class outClass) throws ActivitySerializerException, JsonProcessingException {
+
+        Object result = null;
+
+        if( outClass.equals( Activity.class )) {
+            LOGGER.debug("ACTIVITY");
+            result = facebookPostActivitySerializer.deserialize(mapper.convertValue(event, Post.class));
+        } else if( outClass.equals( Post.class )) {
+            LOGGER.debug("POST");
+            result = mapper.convertValue(event, Post.class);
+        } else if( outClass.equals( ObjectNode.class )) {
+            LOGGER.debug("OBJECTNODE");
+            result = mapper.convertValue(event, ObjectNode.class);
+        }
+
+        // no supported conversion were applied
+        if( result != null ) {
+            count ++;
+            return result;
+        }
+
+        LOGGER.debug("CONVERT FAILED");
+
+        return null;
+
+    }
+
+    public boolean validate(Object document, Class klass) {
+
+        // TODO
+        return true;
+    }
+
+    public boolean isValidJSON(final String json) {
+        boolean valid = false;
+        try {
+            final JsonParser parser = new ObjectMapper().getJsonFactory()
+                    .createJsonParser(json);
+            while (parser.nextToken() != null) {
+            }
+            valid = true;
+        } catch (JsonParseException jpe) {
+            LOGGER.warn("validate: {}", jpe);
+        } catch (IOException ioe) {
+            LOGGER.warn("validate: {}", ioe);
+        }
+
+        return valid;
+    }
+
+    @Override
+    public List<StreamsDatum> process(StreamsDatum entry) {
+
+        StreamsDatum result = null;
+
+        try {
+
+            Object item = entry.getDocument();
+            ObjectNode node;
+
+            LOGGER.debug("{} processing {}", STREAMS_ID, item.getClass());
+
+            if( item instanceof String ) {
+
+                // if the target is string, just pass-through
+                if( String.class.equals(outClass)) {
+                    result = entry;
+                }
+                else {
+                    // first check for valid json
+                    node = (ObjectNode)mapper.readTree((String)item);
+
+                    // since data is coming from outside provider, we don't know what type the events are
+                    // for now we'll assume post
+
+                    Object out = convert(node, Post.class, outClass);
+
+                    if( out != null && validate(out, outClass))
+                        result = new StreamsDatum(out);
+                }
+
+            } else if( item instanceof ObjectNode || item instanceof Post) {
+
+                // first check for valid json
+                node = (ObjectNode)mapper.valueToTree(item);
+
+                // since data is coming from outside provider, we don't know what type the events are
+                // for now we'll assume post
+
+                Object out = convert(node, Post.class, outClass);
+
+                if( out != null && validate(out, outClass))
+                    result = new StreamsDatum(out);
+
+            }
+
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        if( result != null )
+            return Lists.newArrayList(result);
+        else
+            return Lists.newArrayList();
+    }
+
+    @Override
+    public void prepare(Object o) {
+        mapper = new StreamsJacksonMapper();
+        facebookPostActivitySerializer = new FacebookPostActivitySerializer();
+    }
+
+    @Override
+    public void cleanUp() {
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
new file mode 100644
index 0000000..b89a925
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
@@ -0,0 +1,331 @@
+/*
+ * 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 com.facebook.provider;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class FacebookUserInformationProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookUserInformationProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserInformationProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+
+    private Class klass;
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserInformationConfiguration getConfig()              { return facebookUserInformationConfiguration; }
+
+    public void setConfig(FacebookUserInformationConfiguration config)   { this.facebookUserInformationConfiguration = config; }
+
+    protected Iterator<Long[]> idsBatches;
+    protected Iterator<String[]> screenNameBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookUserInformationProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config) {
+        this.facebookUserInformationConfiguration = config;
+    }
+
+    public FacebookUserInformationProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config, Class klass) {
+        this.facebookUserInformationConfiguration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        running.set(true);
+    }
+
+    private void loadBatch(String[] ids) {
+        Facebook client = getFacebookClient();
+        int keepTrying = 0;
+
+        // keep trying to load, give it 5 attempts.
+        //while (keepTrying < 10)
+        while (keepTrying < 1)
+        {
+            try {
+                for( User user : client.getUsers(ids)) {
+                    String json = DataObjectFactory.getRawJSON(user);
+
+                    providerQueue.offer(new StreamsDatum(json));
+//
+                };
+            } catch (FacebookException e) {
+                e.printStackTrace();
+                return;
+            }
+
+        }
+    }
+
+    public StreamsResultSet readCurrent() {
+
+//        Preconditions.checkArgument(idsBatches.hasNext() || screenNameBatches.hasNext());
+
+        LOGGER.info("readCurrent");
+
+        Facebook client = getFacebookClient();
+
+        try {
+            User me = client.users().getMe();
+            ResponseList<Friend> friendResponseList = client.friends().getFriends(me.getId());
+            Paging<Friend> friendPaging;
+            do {
+
+                for( Friend friend : friendResponseList ) {
+
+                    String json;
+                    try {
+                        json = mapper.writeValueAsString(friend);
+                        providerQueue.add(
+                                new StreamsDatum(json)
+                        );
+                    } catch (JsonProcessingException e) {
+//                        e.printStackTrace();
+                    }
+                }
+                friendPaging = friendResponseList.getPaging();
+                friendResponseList = client.fetchNext(friendPaging);
+            } while( friendPaging != null &&
+                    friendResponseList != null );
+
+//                    // Getting Next page
+//                for( Post post : feed ) {
+//                    String json;
+//                    try {
+//                        json = mapper.writeValueAsString(post);
+//                        providerQueue.add(
+//                                new StreamsDatum(json)
+//                        );
+//                    } catch (JsonProcessingException e) {
+//                        e.printStackTrace();
+//                    }
+//                }
+
+//                paging = feed.getPaging();
+//                feed = client.fetchPrevious(paging);
+//            } while( paging != null &&
+//                    feed != null );
+
+
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+        LOGGER.info("Finished.  Cleaning up...");
+
+        LOGGER.info("Providing {} docs", providerQueue.size());
+
+        StreamsResultSet result =  new StreamsResultSet(providerQueue);
+        running.set(false);
+
+        LOGGER.info("Exiting");
+
+        return result;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppId());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getUserAccessToken());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getInfo());
+
+        List<String> screenNames = new ArrayList<String>();
+        List<String[]> screenNameBatches = new ArrayList<String[]>();
+
+        List<Long> ids = new ArrayList<Long>();
+        List<Long[]> idsBatches = new ArrayList<Long[]>();
+
+        for(String s : facebookUserInformationConfiguration.getInfo()) {
+            if(s != null)
+            {
+                String potentialScreenName = s.replaceAll("@", "").trim().toLowerCase();
+
+                // See if it is a long, if it is, add it to the user iD list, if it is not, add it to the
+                // screen name list
+                try {
+                    ids.add(Long.parseLong(potentialScreenName));
+                } catch (Exception e) {
+                    screenNames.add(potentialScreenName);
+                }
+
+                // Twitter allows for batches up to 100 per request, but you cannot mix types
+
+                if(ids.size() >= 100) {
+                    // add the batch
+                    idsBatches.add(ids.toArray(new Long[ids.size()]));
+                    // reset the Ids
+                    ids = new ArrayList<Long>();
+                }
+
+                if(screenNames.size() >= 100) {
+                    // add the batch
+                    screenNameBatches.add(screenNames.toArray(new String[ids.size()]));
+                    // reset the Ids
+                    screenNames = new ArrayList<String>();
+                }
+            }
+        }
+
+
+        if(ids.size() > 0)
+            idsBatches.add(ids.toArray(new Long[ids.size()]));
+
+        if(screenNames.size() > 0)
+            screenNameBatches.add(screenNames.toArray(new String[ids.size()]));
+
+        this.idsBatches = idsBatches.iterator();
+        this.screenNameBatches = screenNameBatches.iterator();
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(facebookUserInformationConfiguration.getOauth().getAppId())
+            .setOAuthAppSecret(facebookUserInformationConfiguration.getOauth().getAppSecret())
+            .setOAuthAccessToken(facebookUserInformationConfiguration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true);
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
new file mode 100644
index 0000000..eae8069
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
@@ -0,0 +1,289 @@
+/*
+ * 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 com.facebook.provider;
+
+import com.facebook.*;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.Post;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookUserstreamProvider implements StreamsProvider, Serializable {
+
+    public static final String STREAMS_ID = "FacebookUserstreamProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserstreamProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig() {
+        return configuration;
+    }
+
+    public void setConfig(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    protected ListeningExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookUserstreamProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookUserstreamProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+
+        executor.submit(new FacebookFeedPollingTask(this));
+        running.set(true);
+    }
+
+    private void loadBatch(String[] ids) {
+        Facebook client = getFacebookClient();
+        int keepTrying = 0;
+
+        // keep trying to load, give it 5 attempts.
+        //while (keepTrying < 10)
+        while (keepTrying < 1) {
+            try {
+                for (User user : client.getUsers(ids)) {
+                    String json = DataObjectFactory.getRawJSON(user);
+
+                    providerQueue.offer(new StreamsDatum(json));
+//
+                }
+                ;
+            } catch (FacebookException e) {
+                e.printStackTrace();
+                return;
+            }
+
+        }
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        StreamsResultSet current;
+
+        synchronized (FacebookUserstreamProvider.class) {
+            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
+            current.setCounter(new DatumStatusCounter());
+            current.getCounter().add(countersCurrent);
+            countersTotal.add(countersCurrent);
+            countersCurrent = new DatumStatusCounter();
+            providerQueue.clear();
+        }
+
+        return current;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet) providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+    }
+
+    protected Facebook getFacebookClient() {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+                .setOAuthAppId(configuration.getOauth().getAppId())
+                .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+                .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+                .setOAuthPermissions(ALL_PERMISSIONS)
+                .setJSONStoreEnabled(true);
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFeedPollingTask implements Runnable {
+
+        FacebookUserstreamProvider provider;
+        Facebook client;
+
+        private Set<Post> priorPollResult = Sets.newHashSet();
+
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
+            provider = facebookUserstreamProvider;
+        }
+
+        @Override
+        public void run() {
+            client = provider.getFacebookClient();
+            while (provider.isRunning()) {
+                try {
+                    ResponseList<Post> postResponseList = client.getHome();
+                    Set<Post> update = Sets.newHashSet(postResponseList);
+                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
+                    Set<Post> entrySet = Sets.difference(update, repeats);
+                    for (Post item : entrySet) {
+                        String json = DataObjectFactory.getRawJSON(item);
+                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                        try {
+                            lock.readLock().lock();
+                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                            countersCurrent.incrementAttempt();
+                        } finally {
+                            lock.readLock().unlock();
+                        }
+                    }
+                    priorPollResult = update;
+                    Thread.sleep(configuration.getPollIntervalMillis());
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
new file mode 100644
index 0000000..b4e5afb
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
@@ -0,0 +1,49 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "properties": {
+        "protocol": {
+            "type": "string",
+            "description": "The protocol"
+        },
+        "host": {
+            "type": "string",
+            "description": "The host"
+        },
+        "port": {
+            "type": "integer",
+            "description": "The port"
+        },
+        "version": {
+            "type": "string",
+            "description": "The version"
+        },
+        "endpoint": {
+            "type": "string",
+            "description": "The endpoint"
+        },
+        "oauth": {
+            "type": "object",
+            "dynamic": "true",
+            "javaType" : "org.apache.streams.facebook.FacebookOAuthConfiguration",
+            "javaInterfaces": ["java.io.Serializable"],
+            "properties": {
+                "appId": {
+                    "type": "string"
+                },
+                "appSecret": {
+                    "type": "string"
+                },
+                "appAccessToken": {
+                    "type": "string"
+                },
+                "userAccessToken": {
+                    "type": "string"
+                }
+            }
+        }
+   }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
new file mode 100644
index 0000000..0454178
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
@@ -0,0 +1,18 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookUserInformationConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "extends": {"$ref":"FacebookConfiguration.json"},
+    "javaInterfaces": ["java.io.Serializable"],
+    "properties": {
+        "info": {
+            "type": "array",
+            "description": "A list of user IDs, indicating users of interest",
+            "items": {
+                "type": "string"
+            }
+        }
+     }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
new file mode 100644
index 0000000..c823a12
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
@@ -0,0 +1,15 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookUserstreamConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "extends": {"$ref":"FacebookConfiguration.json"},
+    "properties": {
+        "pollIntervalMillis": {
+            "type": "integer",
+            "default" : "60000",
+            "description": "Polling interval in ms"
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
index c334f05..23bcb08 100644
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
@@ -2,7 +2,7 @@
     "type": "object",
     "$schema": "http://json-schema.org/draft-03/schema",
     "id": "#",
-    "javaType": "com.facebook.Post",
+    "javaType": "org.apache.streams.facebook.Post",
     "properties": {
         "id": {
             "type": "string"
@@ -69,7 +69,7 @@
             "type": "string"
         },
         "caption": {
-            "type": "boolean"
+            "type": "string"
         },
         "description": {
             "type": "string"
@@ -109,7 +109,8 @@
                         "type": "string"
                     },
                     "created_time": {
-                        "type": "string"
+                        "type": "string",
+                        "format" : "date-time"
                     }
                 }
             }
@@ -163,10 +164,12 @@
             }
         },
         "created_time": {
-            "type": "string"
+            "type": "string",
+            "format" : "date-time"
         },
         "updated_time": {
-            "type": "string"
+            "type": "string",
+            "format" : "date-time"
         },
         "include_hidden": {
             "type": "boolean"
@@ -187,6 +190,14 @@
                     }
                 }
             }
+        },
+        "privacy": {
+            "type": "object",
+            "properties": {
+                "value": {
+                    "type": "string"
+                }
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
new file mode 100644
index 0000000..7f3ab27
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.facebook.test;
+
+import com.facebook.api.FacebookPostActivitySerializer;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.input.BoundedInputStream;
+import org.apache.streams.data.ActivitySerializer;
+import org.apache.streams.facebook.Post;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.pojo.json.Activity;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+
+/**
+* Created with IntelliJ IDEA.
+* User: sblackmon
+* Date: 8/20/13
+* Time: 5:57 PM
+* To change this template use File | Settings | File Templates.
+*/
+public class FacebookActivitySerDeTest {
+
+    private final static Logger LOGGER = LoggerFactory.getLogger(FacebookActivitySerDeTest.class);
+    private FacebookPostActivitySerializer serializer = new FacebookPostActivitySerializer();
+    private ObjectMapper mapper = StreamsJacksonMapper.getInstance();
+
+    @Test
+    public void Tests()
+    {
+        mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, Boolean.TRUE);
+        mapper.configure(DeserializationFeature.ACCEPT_SINGLE_VALUE_AS_ARRAY, Boolean.TRUE);
+        mapper.configure(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT, Boolean.TRUE);
+
+        InputStream is = FacebookActivitySerDeTest.class.getResourceAsStream("/testpost.json");
+        Joiner joiner = Joiner.on(" ").skipNulls();
+        is = new BoundedInputStream(is, 10000);
+        String json;
+
+        try {
+            json = joiner.join(IOUtils.readLines(is));
+            LOGGER.debug(json);
+
+            Post post = mapper.readValue(json, Post.class);
+
+            Activity activity = serializer.deserialize(post);
+
+            LOGGER.debug(mapper.writeValueAsString(activity));
+
+        } catch( Exception e ) {
+            System.out.println(e);
+            e.printStackTrace();
+            Assert.fail();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostActivitySerializerTest.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostActivitySerializerTest.java b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostActivitySerializerTest.java
deleted file mode 100644
index a9b91e9..0000000
--- a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostActivitySerializerTest.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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.facebook.test;
-
-import com.facebook.api.FacebookPostActivitySerializer;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.streams.data.ActivitySerializer;
-import org.apache.streams.data.util.JsonUtil;
-import org.apache.streams.pojo.json.Activity;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.regex.Pattern;
-
-import static java.util.regex.Pattern.matches;
-import static org.hamcrest.CoreMatchers.*;
-import static org.junit.Assert.assertThat;
-
-public class FacebookPostActivitySerializerTest {
-    Node fields;
-    JsonNode json;
-    ActivitySerializer serializer = new FacebookPostActivitySerializer();
-    ObjectMapper mapper;
-
-    @Before
-    public void setup() throws IOException {
-        json = JsonUtil.getFromFile("classpath:org/apache/streams/data/Facebook.json");
-        fields = discover(json);
-
-        mapper = new ObjectMapper();
-        mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, Boolean.FALSE);
-        mapper.configure(DeserializationFeature.ACCEPT_SINGLE_VALUE_AS_ARRAY, Boolean.TRUE);
-        mapper.configure(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT, Boolean.TRUE);
-    }
-
-    @Test
-    public void loadData() throws Exception {
-        for (JsonNode item : json) {
-            Activity activity = serializer.deserialize(getString(item));
-            assertThat(activity, is(not(nullValue())));
-            assertThat(activity.getActor(), is(not(nullValue())));
-            assertThat(matches("id:facebook:people:[a-zA-Z0-9]*", activity.getActor().getId()), is(true));
-            assertThat(activity.getActor().getDisplayName(), is(not(nullValue())));
-            assertThat(activity.getObject(), is(not(nullValue())));
-            if(activity.getObject().getId() != null) {
-                assertThat(matches("id:facebook:[a-z]*s:[a-zA-Z0-9]*", activity.getObject().getId()), is(true));
-            }
-            assertThat(activity.getObject().getObjectType(), is(not(nullValue())));
-            assertThat(activity.getContent(), is(not(nullValue())));
-            assertThat(activity.getProvider().getId(), is(equalTo("id:providers:facebook")));
-            System.out.println(activity.getPublished());
-        }
-    }
-
-
-
-
-    public Node discover(JsonNode node) {
-        Node root = new Node(null, "root");
-        if (node == null || !node.isArray()) {
-            throw new RuntimeException("No data");
-        } else {
-            for (JsonNode item : node) {
-                mapNode(root, item);
-            }
-        }
-        //printTree(root, "");
-        //printUniqueFields(root);
-        return root;
-    }
-
-
-    private String getString(JsonNode jsonNode)  {
-        try {
-            return new ObjectMapper().writeValueAsString(jsonNode);
-        } catch (JsonProcessingException e) {
-            throw new RuntimeException(e);
-        }
-    }
-    private void printUniqueFields(Node root) {
-        Map<String, Set<String>> fieldsByType = new HashMap<String, Set<String>>();
-        fieldsByType.put("objectType", new HashSet<String>());
-        for(Node child : root.getChildren().values()) {
-           for(Node grandChild : child.getChildren().values()) {
-               fieldsByType.get("objectType").add(grandChild.getName());
-               addUniqueValues(grandChild, fieldsByType);
-           }
-        }
-        for(Map.Entry<String, Set<String>> entry : fieldsByType.entrySet()) {
-            System.out.println(entry.getKey());
-            List<String> value = new ArrayList<String>(entry.getValue());
-            Collections.sort(value);
-            for(String val : value) {
-                System.out.println("      " + val);
-            }
-            System.out.println();
-            System.out.println();
-        }
-    }
-
-    private void addUniqueValues(Node child, Map<String, Set<String>> fieldsByType) {
-        if(!fieldsByType.containsKey(child.getName()) && !isNumber(child.getName())) {
-            fieldsByType.put(child.getName(), new HashSet<String>());
-        }
-        for(Map.Entry<String, Node> gc : child.getChildren().entrySet()) {
-            if(!isNumber(gc.getKey()))
-                fieldsByType.get(child.getName()).add(gc.getKey());
-            addUniqueValues(gc.getValue(), fieldsByType);
-        }
-    }
-
-    private boolean isNumber(String key) {
-        Pattern p = Pattern.compile("[0-9]*");
-        return p.matcher(key.trim()).matches();
-    }
-
-    private void printTree(Node node, String spacer) {
-        System.out.println(String.format("%s %s (%s)", spacer, node.getName(), node.getType()));
-        List<Node> children = new ArrayList<Node>(node.getChildren().values());
-        Collections.sort(children);
-        for(Node child : children) {
-            printTree(child, spacer + "      ");
-        }
-    }
-
-    private void mapNode(Node parent, JsonNode jsonNode) {
-        for (Iterator<Map.Entry<String, JsonNode>> iter = jsonNode.fields(); iter.hasNext(); ) {
-            Map.Entry<String, JsonNode> property = iter.next();
-            Node current;
-            String key = property.getKey();
-            JsonNode value = property.getValue();
-            if (!parent.getChildren().containsKey(key)) {
-                current = new Node(null, key);
-                current.setType(value.getNodeType().toString());
-                parent.getChildren().put(key, current);
-            } else {
-                current = parent.getChildren().get(key);
-            }
-            if(!value.isArray() && value.isObject()){
-                mapNode(current, value);
-            }
-        }
-    }
-
-    private static class Node implements Comparable<Node>{
-        Node parent;
-        String name;
-        String type;
-        Map<String, Node> children = new HashMap<String, Node>();
-
-        private Node(Node parent, String name) {
-            this.parent = parent;
-            this.name = name;
-        }
-
-        private Node getParent() {
-            return parent;
-        }
-
-        private void setParent(Node parent) {
-            this.parent = parent;
-        }
-
-        private String getName() {
-            return name;
-        }
-
-        private void setName(String name) {
-            this.name = name;
-        }
-
-        private Map<String, Node> getChildren() {
-            return children;
-        }
-
-        private void setChildren(Map<String, Node> children) {
-            this.children = children;
-        }
-
-        private String getType() {
-            return type;
-        }
-
-        private void setType(String type) {
-            this.type = type;
-        }
-
-        @Override
-        public int compareTo(Node node) {
-            return this.name.compareTo(node.name);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostSerDeTest.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostSerDeTest.java b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostSerDeTest.java
index 4457ed3..6a761cb 100644
--- a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostSerDeTest.java
+++ b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookPostSerDeTest.java
@@ -23,7 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Joiner;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
-import com.facebook.Post;
+import org.apache.streams.facebook.Post;
+import org.apache.streams.jackson.StreamsJacksonMapper;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Ignore;
@@ -42,8 +43,7 @@ import java.io.InputStream;
 public class FacebookPostSerDeTest {
 
     private final static Logger LOGGER = LoggerFactory.getLogger(FacebookPostSerDeTest.class);
-    //private ActivitySerializer serializer = new TwitterJsonActivitySerializer();
-    private ObjectMapper mapper = new ObjectMapper();
+    private ObjectMapper mapper = StreamsJacksonMapper.getInstance();
 
     @Ignore
     @Test


[07/15] git commit: if no ids are present, poll the logged-in user

Posted by sb...@apache.org.
if no ids are present, poll the logged-in user


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

Branch: refs/heads/master
Commit: e0cb5ec25c6a58b45684b77935ddd57ec503a592
Parents: e8511ad
Author: sblackmon <sb...@w2odigital.com>
Authored: Mon Jul 21 15:09:46 2014 -0500
Committer: sblackmon <sb...@w2odigital.com>
Committed: Mon Jul 21 15:09:46 2014 -0500

----------------------------------------------------------------------
 .../provider/FacebookUserstreamProvider.java    | 24 ++++++++++++++------
 1 file changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/e0cb5ec2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
index af7868b..b0bf082 100644
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
@@ -142,10 +142,17 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
             for( String id : configuration.getInfo()) {
                 executor.submit(new FacebookFeedPollingTask(this, id));
             }
+            running.set(true);
         } else {
-            executor.submit(new FacebookFeedPollingTask(this));
+            try {
+                String id = client.getMe().getId();
+                executor.submit(new FacebookFeedPollingTask(this, id));
+                running.set(true);
+            } catch (FacebookException e) {
+                LOGGER.error(e.getMessage());
+                running.set(false);
+            }
         }
-        running.set(true);
     }
 
     public StreamsResultSet readCurrent() {
@@ -279,14 +286,12 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
             while (provider.isRunning()) {
                 ResponseList<Post> postResponseList;
                 try {
-                    if( id != null )
-                        postResponseList = client.getFeed(id);
-                    else
-                        postResponseList = client.getHome();
+                    postResponseList = client.getFeed(id);
 
                     Set<Post> update = Sets.newHashSet(postResponseList);
                     Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
                     Set<Post> entrySet = Sets.difference(update, repeats);
+                    LOGGER.debug(this.id + " response: " + update.size() + " previous: " + repeats.size() + " new: " + entrySet.size());
                     for (Post item : entrySet) {
                         String json = DataObjectFactory.getRawJSON(item);
                         org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
@@ -299,9 +304,14 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
                         }
                     }
                     priorPollResult = update;
-                    Thread.sleep(configuration.getPollIntervalMillis());
                 } catch (Exception e) {
                     e.printStackTrace();
+                } finally {
+                    try {
+                        Thread.sleep(configuration.getPollIntervalMillis());
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
                 }
             }
         }


[02/15] added support for FB user stream and user profile collection

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/test/resources/org/apache/streams/data/Facebook.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/resources/org/apache/streams/data/Facebook.json b/streams-contrib/streams-provider-facebook/src/test/resources/org/apache/streams/data/Facebook.json
deleted file mode 100644
index 3b13611..0000000
--- a/streams-contrib/streams-provider-facebook/src/test/resources/org/apache/streams/data/Facebook.json
+++ /dev/null
@@ -1,251 +0,0 @@
-[{"note":{"from":{"name":"Lauren Kay Bruce","id":"1795508919"},"application":{"name":"Facebook for iPhone","namespace":"fbiphone","id":"6628568379"},"id":"1795508919_4602522719840","created_time":"2013-08-27T23:36:39+0000","type":"status","updated_time":"2013-08-28T00:40:42+0000","privacy":{"value":""},"message":"Can Axe please make a deodorant for her, now."}},
-{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Emmanuel Fernandez Garnica","id":"100000114951549"},"id":"603849322999439_5966940","created_time":"2013-08-28T00:48:32+0000","message":"Que linea para delgada jaja"}},
-{"note":{"from":{"name":"Tevri Bagus P","id":"100002109812180"},"application":{"name":"Facebook for Every Phone","id":"139682082719810"},"id":"100002109812180_519296974817292","created_time":"2013-08-28T00:29:32+0000","type":"status","updated_time":"2013-08-28T00:29:32+0000","privacy":{"value":""},"message":"Yang ax btuh cnta mu seperti cnta ax yg besar buat kamu,\nyang ax syang mu seperti rsa syang ax buat kamu yg tulus,,\nax ga btuh yg lain selain itu doank,,"}},
-{"note":{"comments":{"data":[{"like_count":7,"user_likes":false,"can_remove":false,"from":{"name":"Patrik PipoPupy Tre?o","id":"100000141145825"},"id":"578610008862349_92416511","created_time":"2013-08-28T00:07:12+0000","message":"Uz ma zacinate jebat ...."},{"like_count":1,"user_likes":false,"can_remove":false,"from":{"name":"Duc Anh Le","id":"100004707488552"},"id":"578610008862349_92416523","created_time":"2013-08-28T00:11:33+0000","message":"J� bych toho Balea u? pustil k vod? ..."},{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Luko Eles","id":"100004483267342"},"id":"578610008862349_92416559","created_time":"2013-08-28T00:26:02+0000","message":"T�to spr�va je 2 dni star� :D :D"},{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"David Doktor","id":"100000140869004"},"id":"578610008862349_92416547","created_time":"2013-08-28T00:21:38+0000","message":"Kde berete tyhle zpr�vy?"},{"like_count":0,"user_likes":false,"can_remove":fa
 lse,"from":{"name":"Ondra's Berryss�n","id":"100004401523860"},"id":"578610008862349_92416556","created_time":"2013-08-28T00:25:05+0000","message":"Kone?n? snad!! :)"}],"paging":{"cursors":{"after":"MQ==","before":"NQ=="}}},"from":{"name":"Real-madrid.cz (Official)","category":"Website","id":"165574650165889"},"id":"165574650165889_578610008862349","created_time":"2013-08-28T00:05:51+0000","type":"status","updated_time":"2013-08-28T00:28:55+0000","privacy":{"value":""},"message":"Nejnov?j?� zpr�vy (Sky Sports Italia): Bale p?ijde za 87 milion? eur, Ronaldo z?stane nejdra??� v historii. Uvid�me, uvid�me. Bylo u? t� s�gy dost.","likes":{"data":[{"name":"Luciferus Milanovi?","id":"1423483222"},{"name":"Miroslav ?mok","id":"100000821972410"},{"name":"Denis Pichler","id":"100000222507112"},{"name":"Radek Han�k","id":"1231293572"},{"name":"Lukas Filcik","id":"100003190828171"},{"name":"Radek Tr??a Truhl�?","id":"100000263879140"},{"name":"F�fa Michi","id":"1000019429
 13726"},{"name":"Radim Macoszek","id":"100000411026577"},{"name":"Jan Han�sek","id":"100000467022164"},{"name":"Luk�?ek Pa?il","id":"1304584616"},{"name":"Maros Hric","id":"1794470539"},{"name":"Stanley Uli?ka-Ronaldo Hrica","id":"1180391866"},{"name":"Tom�? Svoboda","id":"1652694973"},{"name":"Dusan Gbelec","id":"100001729069935"},{"name":"Veronica Hrozov�","id":"100000021889547"},{"name":"Jakub Gelnar","id":"100000634354601"},{"name":"Milan Luk�?","id":"100000894167126"},{"name":"Hanina Vr�nov�","id":"100003307243536"},{"name":"Tom�? M�?o Medve?","id":"100000007192762"}],"paging":{"cursors":{"after":"MTAwMDAwMDA3MTkyNzYy","before":"MTQyMzQ4MzIyMg=="}}}}},
-{"comment":{"like_count":7,"user_likes":false,"can_remove":false,"from":{"name":"Patrik PipoPupy Tre?o","id":"100000141145825"},"id":"578610008862349_92416511","created_time":"2013-08-28T00:07:12+0000","message":"Uz ma zacinate jebat ...."}},
-{"comment":{"like_count":1,"user_likes":false,"can_remove":false,"from":{"name":"Duc Anh Le","id":"100004707488552"},"id":"578610008862349_92416523","created_time":"2013-08-28T00:11:33+0000","message":"J� bych toho Balea u? pustil k vod? ..."}},
-{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Luko Eles","id":"100004483267342"},"id":"578610008862349_92416559","created_time":"2013-08-28T00:26:02+0000","message":"T�to spr�va je 2 dni star� :D :D"}},
-{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"David Doktor","id":"100000140869004"},"id":"578610008862349_92416547","created_time":"2013-08-28T00:21:38+0000","message":"Kde berete tyhle zpr�vy?"}},
-{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Ondra's Berryss�n","id":"100004401523860"},"id":"578610008862349_92416556","created_time":"2013-08-28T00:25:05+0000","message":"Kone?n? snad!! :)"}},
-{"photo":{"name":"Timeline Photos","story":"Marco Aur�lio Aur�lio shared FanatiCorinthians's photo.","from":{"name":"Marco Aur�lio Aur�lio","id":"100000629625325"},"application":{"name":"Links","id":"2309869772"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"0":[{"name":"Marco Aur�lio Aur�lio","id":"100000629625325","type":"user","length":21,"offset":0}],"29":[{"name":"FanatiCorinthians","id":"178543808920097","type":"page","length":17,"offset":29}]},"id":"100000629625325_628414700522845","created_time":"2013-08-28T00:47:52+0000","type":"photo","caption":"Recomendo: V�rus Corinthiano\r\nRecomendo: � o Time Do Povo, � o Coring�o\r\nRecomendo: CorinthiAMO","updated_time":"2013-08-28T00:47:52+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=468742883233520&set=a.178546862253125.39468.178543808920097&type=1","object_id":"468742883233520","message":"bic de ponta grossa , kkkkkkkkkkk","properties":[{"href
 ":"http://www.facebook.com/FanatiCorinthiansOficial?ref=stream","name":"By","text":"FanatiCorinthians"}],"picture":"http://photos-f.ak.fbcdn.net/hphotos-ak-ash3/1236687_468742883233520_1910998772_s.jpg"}},
-{"note":{"from":{"name":"Wellington Flores","id":"100004401480520"},"application":{"name":"Facebook for Android","namespace":"fbandroid","id":"350685531728"},"id":"100004401480520_222258177930897","created_time":"2013-08-28T00:47:39+0000","type":"status","updated_time":"2013-08-28T00:47:39+0000","privacy":{"value":""},"message":"Em Pensaar ..!!     Quee por apenas um diaa .  !    Iria ser assim  pra senpre.. .. !  \nSomoss convidados...!!  Aa entrar eem uma viidaa .. !!  Ee ninguem entraa  de bic�o ...!!"}},
-{"video":{"name":"Best inspirational love quotes","from":{"name":"Sajjad Khan","id":"100000877841588"},"application":{"name":"Share_bookmarklet","id":"5085647995"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yj/r/v2OnaTyTQZE.gif","id":"100000877841588_590446897661233","created_time":"2013-08-28T00:47:24+0000","type":"video","caption":"www.youtube.com","updated_time":"2013-08-28T00:47:24+0000","privacy":{"value":""},"description":"Best quotes and saying about love embed in beautiful roses photos, inspiration love quotes and pictures, best quotes about love, cute love quotes. Visit: htt...","link":"http://www.youtube.com/watch?v=e3VPZepU6V0&sns=fb","source":"http://www.youtube.com/v/e3VPZepU6V0?version=3&autohide=1&autoplay=1","message":"its for you bic\r\n","picture":"http://external.ak.fbcdn.net/safe_image.php?d=AQDFvLV-yo-lT-CO&w=130&h=130&url=http%3A%2F%2Fi1.ytimg.com%2Fvi%2Fe3VPZepU6V0%2Fhqdefault.jpg%3Ffeature%3Dog"}},
-{"note":{"from":{"name":"Um Dia de F�ria","category":"Community","id":"201691339996455"},"id":"201691339996455_210984119067177","created_time":"2013-08-28T00:46:35+0000","type":"status","updated_time":"2013-08-28T00:46:35+0000","privacy":{"value":""},"message":"Um dia de F�ria 17 - A Magrela da Oric� e a Bala Perdida\n-------------------------------------------------------------------\n\nEra o s�bado do desfile das campe�s do Carnaval de 1994, ainda tinha uma ressaca do Carnaval para curtir no Largo do Bic�o. Meu amigo Junior MCGyver havia me chamado para ir a Br�s de Pina para ficar com umas meninas e eu fui, claro. Mas chegando l� s� tinha uma e ele n�o quis compartilhar, com a promessa de que o que era meu estava guardado. Mais de uma hora depois e j� cansado de espera-lo a dar uns pegas na madame, voltamos para a Vila da Penha, onde nos encontramos com os demais delinquentes da �poca: Gustavo, Alisson, Luciano, Galdino e por a� vai.\nAo chegarmos na fes
 ta, ele me apresentou a famosa Magrela da Oric� que eu nem preciso citar o nome. Quando eu me deparei com situa��o, confesso que me senti, no m�nimo, surpreso, pois nunca havia conhecido na vida uma menina t�o magra quanto aquela. Ela j� tratou de se apresentar e passou a me chamar de fofo e mostrou total interesse em mim e passou a me perseguir durante toda a festa. Estava me sentindo a verdadeira v�tima de um psicopata Serial Killer. Quanto mais eu despistava, mais ela me encontrava. O grande problema � que na adolesc�ncia, quanto mais o tempo passa mais voc� bebe e quanto mais voc� bebe mais gostosa voc� come�a a achar as mulheres, qualquer uma delas. \nN�o sei hoje, mas nos anos 90 era muito comum ter carros cheio de batidas no caput, e voc� pagava uma miserinha por uma garrafa e enchia a cara a noite inteira, sendo que voc� nunca bebia sozinho, pois os amigos sempre aparecia com um copinho descart�vel para ser arregrado.\nE ao bater das doze ba
 daladas noturnas, a carruagem virou ab�bora e as minhas resist�ncias viraram p�. A Magrela pegou na minha m�o e disse: � agora!!!! Me aproximei e mais uma vez consegui resistir aos seus poderes hipn�ticos,,, s� n�o sabia at� quando. Passava desfilando com ela e todos me homenageavam pela coragem e bravura e ela come�ou a ficar impaciente e disse que se eu n�o queria, bastava falar,,,, mas fui um fraco e ao mesmo tempo um cavalheiro, pois n�o poderia deixar a menina magoada. Ent�o bolei um plano mirabolante. O Alan e a sua irm� estavam indo embora, ent�o falei pra ela que far�amos companhia para os dois e depois selar�amos o esperado beijo da noite. Era o local perfeito pra n�o ser flagrado por toda sociedade Viladapenhana ou ent�o o momento mais prop�cio para sair correndo.\nFoi quando rolou um tumulto violento e uma correria danada,,, todos desembestados e eu sem saber o que estava acontecendo,,, ouvi tiros, gritos e a sensa��o mais louca
  que eu j� senti na minha vida, tentei correr mais levei uma porrada na nuca, ouvi som de trinca-ferro e os olhos come�aram a triplicar as imagens. O som ficou mais lento, como se reduzisse a rota��o do disco de vinil. Olhava para os lados e via pessoas ainda correndo. Tentei dar o pr�ximo passo, mas n�o tinha mais for�a nas pernas. Senti o ch�o chegando muito vagarosamente, coloquei meus bra�os para amortecer a queda e ao tocar o asfalto, como num passe de m�gica, tudo voltou ao normal. Eu ainda tentei rastejar para fugir do tiroteio, mas um transeunte me segurou e me informou aquilo que eu n�o queria pensar: Eu havia levado um tiro e sangrava muito. Fui virado pra cima e vi as pessoas se amontoarem em cima de mim. Rasgaram minha blusa para arejar, tiraram meus t�nis para arejar, e por sinal at� hoje eu n�o os vi de volta, um Nike novinho, do Alisson... \nO boato dos baleados se espalhou e os carniceiros dos meus amigos vieram correndo pra ver quem era o
  bandido alvejado e se deparam comigo deitado sobre um mar de sangue. Todos se apavoraram e o Alisson, no seu melhor estilo capoeira, furou o cord�o de isolamento e veio ficar comigo. Eu estava com um medo fodido de morrer, olhava pro lado e via o sangue escorrer pela rua abaixo. A �nica viatura de pol�cia que havia no local,  levou um baleado na garganta e um gr�vida alvejada de rasp�o na barriga. E eu fiquei l�, esperando a minha hora chegar. Foi quando um santo taxista se ofereceu pra me levar ao hospital. Ao tentar levantar, eu nunca poderia imaginar que a minha cabe�a pesava tanto, acho que a bala rompeu alguns ligamentos de sustenta��o dos m�sculos do pesco�o. \nEntrei no carro com muito sacrif�cio e sentia vertigens incontrol�veis, iria desfalecer ou mesmo falecer a qualquer momento, o problema � que a luz branca n�o veio, ent�o pensei: Fudeu, vou pro inferno. O Alisson tentava me animar, me fazia cantar Legi�o Urbana: Que Legi�o Urbana �
  o caralho,,, estou morrendo cara, voc� n�o est� vendo???.... E os olhos temiam em fechar. O hospital Get�lio Vargas nunca foi t�o longe,,, levamos uma eternidade para chegar,,,, essa era a sensa��o. Ao chegar no hospital, n�o tinha leito, me deitaram numa maca met�lica fria, sem camisa e com hipotermia, por causa da perda de sangue. Fui operado ali mesmo, extra�ram a bala, embalaram numa gaze e me deram. Me senti uma mam�e no parto ao ter seu filho nos bra�os. Olhei pra bala e disse: Filha da Puta,,, quase me matou.\nE pra fechar a noite com chave de ouro, o Alisson me pergunta: \n- E a magra, pegou?\n- N�o, escapei,,,, ufa... santa bala perdida!!!!"}},
-{"note":{"from":{"name":"Dakoda Mandujano","id":"100002494114668"},"application":{"name":"Mobile","id":"2915120374"},"id":"100002494114668_501954526564347","created_time":"2013-08-28T00:45:48+0000","type":"status","updated_time":"2013-08-28T00:47:43+0000","privacy":{"value":""},"message":"i need my Bic sucked right now lol"}},
-{"note":{"from":{"name":"Gi?t M?c M�a H�","id":"100006547540821"},"id":"100006547540821_1389297757965129","created_time":"2013-08-28T00:41:31+0000","type":"status","updated_time":"2013-08-28T00:41:31+0000","privacy":{"value":""},"message":"Ch?c ch?n h?n anh to�n m?c t�m \nB� kh�ng tin ch?m th? v�i l?n \nM?t l� ???c th?y cho m??i ?i?m \nHai l� tim b� b? ... b�ng khu�ng\nAnh l�m th? to�n b?ng vi?t bic \nL�m quen ch?a bi?t vi?t b?ng g� \nS?c nh? ng�y x?a c�n l? m?c \nS�n tr??ng t�m ??m m?t c�y si\n\nB�i h�t n�o \"s?i th??ng s?i nh?\" \nC�n th? anh \"gi?t nh? gi?t th??ng\" \nM?t h�m anh bi?n th�nh quy?n v? \nT�nh nguy?n y�u gi?t m?c t�m ?ang bu?n"}},
-{"note":{"from":{"name":"Trang T�mat?","id":"100002216264738"},"application":{"name":"Facebook for iPhone","namespace":"fbiphone","id":"6628568379"},"id":"100002216264738_499056076844930","created_time":"2013-08-28T00:40:48+0000","type":"status","updated_time":"2013-08-28T00:40:48+0000","privacy":{"value":""},"message":"Tinh m? s??ng ?i t?p e r� bic\n...c�i c?m gi�c t?p xong ng ??t h?t m? h�i r�i lao xe vun v�t => m�t r??i iii\n=> th?t l� th�ch >:'D<"}},
-{"note":{"from":{"name":"Richly Awthentic","id":"100001409130885"},"id":"100001409130885_589039464486353","created_time":"2013-08-28T00:34:26+0000","type":"status","updated_time":"2013-08-28T00:34:26+0000","privacy":{"value":""},"message":"fire in the HOLE!..*stirkes BIC*"}},
-{"note":{"from":{"name":"Black Friday Cyber Monday on Sale","category":"Product/service","id":"1376926415869828"},"id":"1376926415869828_1382427048653098","created_time":"2013-08-28T00:34:10+0000","type":"status","updated_time":"2013-08-28T00:34:10+0000","privacy":{"value":""},"message":"BIC Sport Adjustable Stand Up Paddle Blade\n\nCHECK TO FIND LOW PRICE >> http://tinyurl.com/gbkj4qsx/B009VYHJRI\n\nBIC Sport Adjustable Stand Up Paddle Blade The four Most Well-known Jugglers At any time - Find out From Them, YOU Could Be Subsequent! Have you at any time dreamed of remaining the future legend in juggling? I have. So I wrote this manual on the four most famed jugglers, and their insider secrets to turning out to be the legends they had been. Understand from all those insider secrets, and you can be the following legend in juggling! BIC Sport Adjustable Stand Up Paddle Blade.  The Journey Privilege of Motorized Paragliding Paragliding experts already have the liberty to fly like birds
  on their have at their own prices and travel as much as they can. They appreciate the privilege of touring or crossing nations. BIC Sport Adjustable Stand Up Paddle Blade.  Lacrosse Drills - Shooting Drills From Cornell Lacrosse These are great lacrosse drills for increasing the capturing skills of your lacrosse players. Lacrosse coaches can make their teams much more powerful at scoring additional plans with these easy drills. These drills came in an job interview with Coach Tambroni from Cornell Lacrosse."}},
-{"note":{"from":{"name":"Cyber Monday Black Friday Online","category":"Product/service","id":"565321923532890"},"id":"565321923532890_568015023263580","created_time":"2013-08-28T00:33:37+0000","type":"status","updated_time":"2013-08-28T00:33:37+0000","privacy":{"value":""},"message":"BIC Beach Kayak Paddle (1 Piece)\n\nCONTINUE TO SEE LOW PRICE >> http://tinyurl.com/rerqq91g/B004XFVOWG\n\nBIC Beach Kayak Paddle (1 Piece) Yukon one thousand - The Longest Canoe and Kayak Race The Yukon a thousand is the longest canoe and kayak race in the earth. Racers contend with seven to eight days of steady paddling with only a few hrs of rest each individual night time. BIC Beach Kayak Paddle (1 Piece).  Athletics - Bringing Americans With each other Sports have touched and have achieved out to each and every side of American existence. Due to the greater availability of medium made use of to observe and share these activities, (Tv set and the Internet), sporting activities encounters and reminis
 cences have been shared a lot more and with hundreds of thousands of Americans in 1 variety or another. We have found dramatic wins, losses, championships, terrific plays, and not so terrific performs. People adore sports activities and the athletics that perform them and really like to share and communicate about these good moments. BIC Beach Kayak Paddle (1 Piece).  The Decline of Cannabis Details Hashish has been the most morally grey topic place for a long time and the information and facts you acquire on the subject can be really far from the reality. Certainly in most nations spherical the planet Hashish is Unlawful, however individuals are even now working with it so there really should be a bank of details about it."}},
-{"note":{"from":{"name":"Gabriel Paul Pierce-Lackey","id":"100003952643883"},"id":"100003952643883_293782680763484","created_time":"2013-08-28T00:30:39+0000","type":"status","updated_time":"2013-08-28T00:38:29+0000","privacy":{"value":""},"message":"If you snag lighters, I hate you"}},
-{"note":{"from":{"name":"Udin Zeoss","id":"100004916228419"},"application":{"name":"Mobile","id":"2915120374"},"id":"100004916228419_193897570784116","created_time":"2013-08-28T00:30:09+0000","type":"status","updated_time":"2013-08-28T00:30:09+0000","privacy":{"value":""},"place":{"name":"PaBriK paNg UjuNg Na ?? BiC-PurwakarTa","location":{"city":"Cikampek","latitude":-6.440615,"zip":"","country":"Indonesia","street":"","longitude":107.4492325},"id":"469609653049663"},"message":"Santai dl sblm gwe"}},
-{"note":{"from":{"name":"Andrel Barbour Doing Her","id":"100000828229524"},"application":{"name":"Mobile","id":"2915120374"},"id":"100000828229524_567828116588124","created_time":"2013-08-28T00:29:53+0000","type":"status","updated_time":"2013-08-28T00:48:36+0000","privacy":{"value":""},"message":"ALL GLORY....I DNT CARE...OR GIVE A FLYIN FLOCK WHAT NOBODY ELSE DO OR SAY AS LONG MY GLORY NAME ANIT IN IT....1 BIRD DNT KILL A STONE...AND I THAT BIC BIRD AND GOING OR WORRYIN ABOUT NOTHING N MY LIL BRO..EVERETTE..BKA...DANK VOICE...SAY IT AGAIN...I ANIT WORRYIN ABOUT NOTHING....OW OW OW OW OW OW...JUST BRING THE SMOKES I GOT THE WRAP...BRING THE ALCOHOL...I GOT THE CUPS...BRING THE BARBQUE I GOT THE PLATES...FOLKS...SPOONS AND NAPKINS...CANT GET NO MATTER THEN THAT....LMAO..."}},
-{"note":{"from":{"name":"Blackie Ibrahim","id":"100001611878761"},"application":{"name":"Mobile","id":"2915120374"},"id":"100001611878761_569548039775587","created_time":"2013-08-28T00:29:26+0000","type":"status","updated_time":"2013-08-28T00:29:26+0000","privacy":{"value":""},"message":"THIS ARE DAYZ 2  BE REMEMBRED!!!!\n\n  I miss d days when we went to\nschool, lined up & D headmistress &\nteachers inspect our nails & uniform\n& den we match to our classrooms.\n? D days of Natco Biscuit, Iced\nColored water tied in nylon we called\nit \"kulaid\"\n? D days of pehpeh mint\n? D days of WAN POT,correction corner(paddy nor piss dae!)\n? Hay, i remember those days when\none block coin na money,when\ngroundnut was 50 leone.&\nsweet was 2,3,4 for 50 Leone\n? days of messing game,forl shit(police,find me d thief), police and thief\n? D days of mummy & daddy play\n? days when we use to build houses\nwith sand, play borlorgie game, \nskipping, street ball stop! U don remember!\n?those days 
 wen we used to fly kite\non streets, wen boys used to use d\nMilk cups to make cars & their\ndaddy's hanger as d steering\n?those days when rubber band(fark) was\nstock exchange\n?days when mortal kombat,street fighter,snake and the monkey shadow,power rangers, spider man were our\nfavourite movies\n? D days of Pepsi soft drinks & milo and maltina advert on black and white tv\n? when we say ''ar stiff,nor stiff o. Wan cup wata! '' and slapping our colleagues saying 'fresh leg o'\n? Those days when bic pen\nwas d best\nD days wen we used to drink water\nfrom d tap even suck out d water if its\nnot coming out\n? D days when SLBS will show\nrainbow color for 30 mins then\nnational anthem before they resume\nprogram @ 4pm\n? D days wen silectto were d best footwares\n?Days wen we all sing ''neneh binta dae sell chi gum, chocolate,die mint,sweet!\nI'm really\nproud to have experienced\nall this. If u r nt smiling it means u\nwere nt born in my generation.\nAdd yours and pass the fun arou
 nd, hey heee haahaaaaaa opopinaaaazzzzz"}},
-{"note":{"from":{"name":"William George","id":"100003820913189"},"application":{"name":"Facebook for Android","namespace":"fbandroid","id":"350685531728"},"id":"100003820913189_320249768112384","created_time":"2013-08-28T00:29:14+0000","type":"status","updated_time":"2013-08-28T00:29:14+0000","privacy":{"value":""},"message":"Felling bad today have no meds for my diabetes because I am broke I need some help I have some u of l tee shirts and hats u of k all so all brand new will sell cheap all so bic  lighters if anyone can help me out hmu on fb any help would be great I feel like I am desperate so ty"}},
-{"photo":{"from":{"name":"Thami de Oliveira","id":"100003470164026"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yz/r/StEh3RhPvjk.gif","id":"100003470164026_390972087695127","created_time":"2013-08-28T00:29:07+0000","type":"photo","updated_time":"2013-08-28T00:44:13+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=390971777695158&set=pcb.390972087695127&type=1&relevant_count=2","object_id":"390971777695158","message":"MENINAS QUE TIVEREM INTERESSE NESSE VESTIDO TEM NA COR BRANCO,PRETO,AMARELO,AZUL BIC,ROSA PINK E AZUL PISCINA , COMENTA COM A COR QUE FOR QUERER PRA EU PODER FAZER A ENCOMENDA, S� VOU TRAZER DE QUEM COMENTAR !!!","picture":"http://photos-a.xx.fbcdn.net/hphotos-ash3/1234956_390971777695158_1843537076_t.jpg"}},
-{"note":{"from":{"name":"Black Friday Cyber Monday on Sale","category":"Product/service","id":"1376926415869828"},"id":"1376926415869828_1382424758653327","created_time":"2013-08-28T00:28:52+0000","type":"status","updated_time":"2013-08-28T00:28:52+0000","privacy":{"value":""},"message":"BIC Sport ACE-TEC Stand-Up Paddleboard\n\nCONTINUE FOR SPECIAL PRICE >> http://tinyurl.com/gbkj4qsx/B005SH9TNO\n\nBIC Sport ACE-TEC Stand-Up Paddleboard Types of Geocaching Caches The satisfaction achieved when enjoying geocaching usually lies on the discovery of what a cache incorporates. This serves as the reward for all the terrain that a cacher has successfully conquered. Caches come in different varieties and usually include a logbook (for greater caches) or log paper (for more compact caches). BIC Sport ACE-TEC Stand-Up Paddleboard.  Leisure at Relatives Reunions Looking for a risk-free outside game that can be performed by folks of all ages and qualities? Then search no further than cornhole.
  Cornhole is a addictive activity that is easy to master and a ton of enjoyable to play. BIC Sport ACE-TEC Stand-Up Paddleboard.  How to Prevent Soaked Ft When Going for walks in the Uk Most of my hiking and strolling usually takes position in the British isles. Locations to stroll this kind of as the Lake District, Scotland and Snowdonia are my favourite spots. These locations are wonderful."}},
-{"note":{"from":{"name":"Fahri  Kuliev","id":"100003757786565"},"id":"100003757786565_336091023192821","created_time":"2013-08-28T00:27:43+0000","type":"status","updated_time":"2013-08-28T00:27:43+0000","privacy":{"value":""},"message":"F?XR? M�SL�M\nBANKS CHA?RMAN\nMini chronologial novel\nBirinci hiss?\n V  \n H?s?n Qas?movi�in yan?nda,  m?n, t?k deyildim. T?fti? ?�b?sinin r?isi d? var idi. H?l? o,  t?fti? ?�b?sinin r?isini dinl?yirdi. T?fti? ?�b?sinin r?isi tutatl? s�but v? d?lill?rl?, sanki �z dedikl?rini �z� d? t?sdiq edirdi. \n O, �z�n� H?s?n Qas?movi�? tutaraq, - bu yolda? birinci yoxlama i?ind? i?tirak edir, akt mateiallar?n? oxudum, birinci d?f? ���n yoxlama materiallar? pis deyil. Ancaq veril?n m?lumata ?sas?n, qeyd edim ki, bu yolda? yerl?rd? tamam ba?qa i?l?rl? m???ul olur. Bu, h?m onun t?fti?�i ad?na, h?m d? i?l?diyi t??kilat?n ad?na l?k? g?tir? bil?r. O, yoxlama d�vr�nd?, ?eir yazmaqla m???ul olub, yerli radio verli?l?rind? g�n
 a??r? �?x??lar edib, rayon q?zetind? ?eirl?r d?rc etdirib. Fikirl??in ki, bu i?l?r bel? davam ed?rs?, onda t?fti? i?i hans? m�sib?tl?rl? �zl??? bil?r, h?m keyfiyy?t c?h?td?n, h?m d? yoxlan?lan t??kilata  qar??  t?l?bd?n v? s. s�hb?t gedir. Ax?, bu bizim i?imizl? daban-dabana zidd bir m?s?l?dir. Sonra yerl?rd?n biz? eham ed?c?kl?r ki, siz bura t?fti?�i g�nd?rmisiniz, yoxsa, jurnalist, yaz?�?, ?air.\n M?n bunlar?n hardan qaynaqland???n? bilirdim v? �z-�z�m? fikirl??irdim ki, i?? girdiyim g�n�n s?h?ri t?fti? ?�b?sinin r?isi,  t??kilat?n, b�t�n rayonlara ged?c?k  m?ktublar?n?, m?n d? daxil olmaqla bir ne�? n?f?rd?n xahi? etdi ki, ba? po�t idar?sin? apar?b t?hvil ver?k. Bu m?s?l?y? m?n etiraz ed?r?k, - t??kilat?n ma??n? yoxdu, n?di, -demi?dim  v? m�?yy?n bir m?saf?y?, �z� d? ??h?rin m?rk?zind?,  ??ll?nib y�k aparma?? �z�m? s????d?rmam??d?m. V? m?n �z pay?ma d�??n m?ktublar?  taksi tutaraq po�ta aparm??d?m, bu da ?�b?d?  m?nd?n gizli s
 �z-s�hb?t? s?b?b olmu?du. �st?lik d? b�t�n bunlar m?nim yek?xanal???m kimi d?y?rl?ndirilmi?di. O, dan??d?qca h?min ?hvalat yad?ma d�?d� v? �zl�y�md? fikirl??dim ki, r?isin ?lin? gir?v? d�?�b,  hay?f?n? bu yolla v? bird?n �?xmaq ist?yir. G�zl?rim onun �z�nd?, qulaqlar?m is? onun gileyind? olmas?na baxmayaraq, sakit oturmu?dum, ��nki h?r ?ey H?s?n Qas?movi�in ver?c?yi q?rardan as?l? oldu?unu, art?q, g�t�r-qoy etmi?dim. Ancaq b�t�n bunlara baxmayaraq, i�imd? bir rahatl?q da var idi v? bu rahatl?q t?k m?nim i�imd?kil?rl? ba?l? deyildi, h?m d? t?fti? ?�b?sinin r?isinin H?s?n Qas?movi�l? olan m?nfi m�nasib?tl?ri il? ba?l? bir m?s?l? idi.\n H?s?n Qas?movi� d?  ona diqq?tl? qulaq asmas?na baxmayaraq, sanki onun ?ikay?t xarakterli gileyin? yox, ?a? atl? da?d?mirin na??l?na? qulaq as?rd?.  T?fti? r?isi art?q hiss etdi ki, dan??ma?a bir ?ey qalmad?, oturdu?u stulda yayxand? v? a??r bir y�k�n alt?ndan azad olmu? kimi, d?rind?n bir n?
 f?s ald?. O, sif?tind? bir ovuc qom olmu?, k�lg? r?ngind? t?b?ss�m� il?  adamda ikrah hissi do?ururdu. O, el? adam? bo?an bu ikrah hissi il? d? arxay?n-arxay?n H?s?n Qas?movi�in �z�n? bax?b qalm??d?. Qalm??d? ki, H?s?n Qas?movi� bu m?s?l?y? nec? yana?acaq. Bird?n-bir?, m?n?  el? g?ldi ki, bu adam canl? deyil, yapon d?zgahlar?nda istehsal olunmu? robotdu v? d�ym? il? avtomatla?d?r?lm?? formada  idar? olunur, laz?m olanda dan???r, laz?m olmayanda susur.\n H?s?n Qas?movi� is? yax?? i? g�rm?di, ona qulaq asandan sonra, onun getm?yin? i?ar? etdi. G�r�n�r o, m?ni r?isin yan?nda ya pis v?ziyy?td? qoymaq ist?mirdi, ya da  fikirl??di ki, m?n d? insanam, bu m?s?l?y? m?nim d? �z yana?mam ola bil?r v? onda t?fti? ?�b?sinin r?isi pis v?ziyy?td? qala bil?r. B?lk? d? o, m?niml? t?kb?t?k dan??maq da, m?ni g?l?c?k i?l?r ���n t?dqiq etm?k ist?yirdi, fikirl??dim. Bu o q?d?r d? �ox �?km?di, H?s?n Qas?movi� sa? �iyini �std? m?n? t?r?f d�nd�: - a ki?i, m?
 n g?lmi??m rayona? rabo�i? paltarda, s?n is? kostyumda otrmusan, bo?az?nda da qalstuk, o q?z da b�yr�nd?n bel? ke�ir, el? ke�ir, sa�lar? da s?nin �iynind? g?z?-g?z?.  S?n d? feyziyab olub, s�z qo?ursan, ?eir yaz?rsan, n? bilim daha n?l?rl? m???ul olursan. Birinci n�vb?d?, o q?z ba? m�hasibdi v? s?n d? onun s?n?dl?rini yoxlay?rsan, bu formada s?n hans? n�qsanlar? �z? �?xarda bil?rs?n. O q?z s?ni bax??lar?yla aldada-aldada ciddi n�qsanlar?, m?nims?m?l?ri d? s?nd?n yay?nd?ra bil?r. Bu, bu rayonda olmasa da, ba?qa bir rayondak? olacaqd?r. M?n s?ni i? g�rm?y? g�nd?rmi??m, s?n d? l?,..l?,..lo,..lo...i?l?rl? m???ulsan.  Bu, b?s el?mir kimi, q?zet bel? getdi, radio bel? getdi. G�rd�n ki, s?h?rd?n t?fti? ?�b?sinin r?isi durub-dayanmadan �y�d�b t�k�r, m?n burda s?n? g�z??t? ged?c?m, ondan i? t?l?b ed?mm?y?c?m, ona g�z??t? gets?m, s?nin ?m?k kitab�an korlanacaq.\n M?n, verilic?k cavablar?m?, i�imd? yava?-yava? g�t�r-qoy edirdim. V? bu
  cavablar? g�t�r-qoy etdiyim? g�r? d? bir q?d?r arxay?nla?m??d?m v? g�zl?yirdim ki, H?s?n Qas?movi� s�z�n� deyib qurtars?n, m?n d? �z d?lil-s�butlar?mla onu inand?r?m ki, - H?s?n Qas?movi�, ax? el? deyil, bu i? m?nim ?sas i?im? mane�ilik etmir, ?ksin?, bu i? d?, o i? d? bir-brini tamamlay?r.\n O, art?q, sar???n v? cod sif?tind? gizl?tdiyi g�y v? z?himli g�zl?rini m?n? zill?mi?di v? m?nim n? dey?c?yimi s?brsizlikl? g�zl?yirdi.\n M?n, ani olaraq �z�m� y????d?rd?m v? n?d?n ba?layaca??m? g�t�r-qoy etm?y? �al??d?m, ��nki qar??mda ya?l? bir insan?n oldu?unu d�?�nd�m. V? bir q?d?r ehtiyatla, inand?r?c? ??kild? onun �z�n? baxd?m, - H?s?n Qas?movi�, -dedim,-yaz?-pozu he� vaxt d?qiq sah?l?r? mane�ilik t�r?tmir, ?ksin?, bir-birin? b?lk? d? dayaq durur, ��nki burda m?n?viyyat �l��l?ri d?, h?yat �l��l?ri d? bir-birini tamamlay?r v? m�hk?m bir ba?lant? yarad?r. Ist?r d�nyada v? ist?rs? d? yax?n ??rq ?d?biyyat?nda 
 taa q?dimd?n yaz?-pozu adamlar? h?mi?? d?qiq v? ciddi i?l?rl? m???ul olmas?na baxmayaraq, ?eirl? d?, yaz?�?l?qla da, b?st?karl?qla da, r?ssaml?qla da, n?qqa?l?qla da v? s. m???ul olublar. B�y�k T�rk imperiyas?n?n ?ah? Sultan Suleyman h?m d? d�vr�n�n yax?? ?airl?rind?n olmu?du. Az?rbaycan imperiyas?n?n yarad?c?s? ?ah ?smay?l X?tai s?rk?rd? v? ?ah olmas?ndan as?l? olmayaraq �z ?airliyi il? Az?rbaycan ?d?biyyat?n?n apar?c?lar?ndan biriydi. Bununla yana??, m?n b�t�n bunlar? sadalamaq da �z�m? haqq qazand?rmaq ist?mir?m. M?s?l? bundad?r ki, m?n b�t�n bu yaz?-pozu m?s?l?si il? i?d?n sonra m???ul oluram. M?n, dan???b qurtard???m? hiss etm?dim, birc? onu hiss etdim ki, bu sar???n ki?inin �z�nd? n?s?, m?n? qar?? bir xo? ?hval-ruiyy?  ?m?l? g?lmi?di. V? m?n bunu, �zl�y�md? raz?l?q hissi kimi, �z�m�n d? q?l?b?m kimi hiss etdim. Bu da??n?q hissl?rimin alt?nda o, he� n? dem?d?n getm?yim? i?ar? etdi. �z�md?n raz? qalsam da i�imd? bir narahatl?q 
 hissi d? yox deyildi v? bu hissl?rl? d? otaqdan �?xd?m.  \n  Sanki �z�m? arxay?n idim, ancaq bu arxay?n�?l???n f?rqini saf-��r�k ed? bilmirdim. Birc? onu bildim ki, t?fti? ?�b?sind? stulumda oturmu?am. T?fti? ?�b?sinin r?isi d? t?fti?�il?rl? hans?sa, m�r?kk?b bir sual?n h?llini ayd?nla?d?r?r. M?n? �atan o oldu ki, s�hb?t, m�hasibat hesablar?ndan birinin hans? hesabla m�xabirl??m?sind?n gedir.  Bel? ba?a d�?d�m ki, hesab?n debetini d�zg�n t?yin  etmi?dil?r, ancaq onun hans? hesab?n kreditin? m�xabirl??m?si m�bahis?li qalm??d?. Donmu? bax??lardan hiss etdim ki, t?fti? ?�b?sinin r?isinin �z� d? bunu d�z-?m?lli bilmir. M?n, �zl�y�md? g�t�r-qoy etm?y? �al??d?m, ancaq m?n d? bu m?s?l?l?rin d?rkind?n �ox uzaqda idim. \n\n VI\n\n Art?q Xank?ndind? idik. �� n?f?r idik. M?n, kadrlar ?�b?sinin i?�isi Namik v? h�quq?�nas ?lqar. Namik t?bi?t?n it oynadan idi. Ilqar?n �z� kimi d? t?bi?ti var idi, haynan hay kimiydi, y
 aynan yay kimiydi, yay?n? �?kib oxunu gizl?dirdi. Kimliyini bilm?y?n bir adam idi. Gah ?sirdi, gah k�s�rd�, gah da as?b k?sirdi, h?mi?? d? adam? aldatma?a �al???rd?. O, m?niml? bir az xo? r?ftar ed?n kimi, h?r ?eyi unudurdum v? onunla da tez raz?la??rd?m, bu da m?n? baha ba?a g?lirdi. Sonra hiss edirdim ki, d�?m?ni hardasa axtarmaqla deyil, d�?m?n el? �z i�imizd?ymi?. Qan qrupu �ox k�p?ko?lu n?sn?ymi?.  Bu t�rkl�k d? bizim ba??m?za b?la olub. Boz qurdu �z�m�z? bayraq etmi?ik, ulam-ulam ulay?r?q, ulaya-ulaya da vaxt apar?r?q, bu vaxt ?rzind? d? hay ba??m?z? k?sib qoyur dizimizin �st�n?.  Arxada qal?b gecik?nl?rmiz is? salidorla yum?altd??? �?km?l?rini s�r�y?-s�r�y?  v? soyuqqanl?, soyuqqanl? yanda, y�r?d? can?n? gir?l?yir.\n Hay olmas?na baxmayaraq Razmik m?nim ���n yax?? adam t?siri ba???lay?rd?. Bu, b?lk? d? m?n? el? g?lirdi. ��nki o, m?nim ���n yax?? adam, haylar ���n pis adama ��vrilmi?di. Fikirl??irdim ki,
  b?lk? qan damar? bizd?ndi. B�t�n bunlar da ehtimal xarakteri da??y?rd?. O, m?n? yalman?b �z�n�nk�l?ri sat?rd?, bu is? m?nd? ba?qa bir t?sir yarad?rd?. Fikirl??dim ki, �z�n�nk�n� satan adam, sabah m?nim �z�m� d? pis v?ziyy?td? qoya bil?r v? ?�k�r edirdim ki, h?l?  onun yan?nda bir q?bah?tim yoxdur. Ikimiz bir otaqda oturmu?duq, onun �z�n? baxd?m, y?ni baxd?m ki, o, el? bel? i?�i deyil, h?r halda bizim ba? idar?mizin ?tat vahidi kimi, m?sul bir v?zif? da??y?r. V? ona g�r?, birinci d?f? oldu?um bu Xank?ndind?ki  t??kilat?n idar? sistemin? aid m?s?l?l?rl? yax?ndan tan?? olma??m ���n onun yard?m?na ehtiyac?m var idi.  O, el? bil m?nim i�imd?n ke�?nl?ri d?rk etmi?di v? ?lind?ki s?n?di stolun �st�n? qoyub, s�z? ba?lad?:\n O, - eli man siza bir ne�a pakt veracam, siz da eli ona uy?un i? gorun ha. Birincisi, siz eli soru?a bilars?n?z ki, ?u?a ?�bas?nda i?layan i?�ilar?n mukafat?n? niy? kas?blar ha. Eli siza qaranl?q qalmas?n deya
  izah edim ha. ?u?a rayonu hesabat? be? g�n gecikdirib ha, ona g�ra da m�kafatlar?n? kas?blar ha, ancaq Qadrut ?�b?si hesabat? on be? g�n gecikdirb ha, ancaq bunlar �z adamlar?d? deya tarixda duzali? ediblar v? hesabat vaxt?nda verilmi? kimi qeydiyyata al?n?b v? saxta yolla da onlara mukafat veriblar ha.\n M?n, bunlar? bilirdim ancaq d?rinl??dirm?k ���n onun �z�n? baxd?m:\n - bunlar? onlara nec? izah edim ki, s?nin informasiyan oldu?unu bilm?sinl?r, -dedim\n Razmik , - eli, siz ba? m�hasibi �a??r?n v? ondan soru?un ha, o da izah edacak ki, el?di, bel?di. Onda siz h?r iki ?�ban?n hesabat?n? tahlil edin ha, onda g�racaks?n?z k?, ?u?an?n gecikma tarixi oldu?u kimi qal?b ha, ancaq Qadrutun hesabat?nda isa lap ammal?-ba?l? duzal?? ediblar ha, v? vaxt?nda qabul edilmi? kimi d? sanadla?d?r?blar ha, rubluk mukafatlar?n? da veriblar ha. Biliram ki, boyunlar?na almayacaqlar ha, ondan sonra Qadruta bir teleqram vurar?q ha, onlar da hesabat?n giri?-�?x?? tarixini
  tasd?q edacaklar ha. Onda s?n bunlar? onlar?n g�zlar?na soxarsan v? ?u?an?n rubluk mukafat?n? da verdiracaksan, ya da Qadrutun mukafat?n? tutduracaqsan ha.\n Razmikin faktlarla dediyi, yoxlaman?n gedi?at?na k�m?yi olsa da, ?lqar?n aldad?c? h?r?k?tl?ri m?ni qane etmirdi. Guya  o, bu i?d? �z�n� d�zg�n,  h�quq?unas  t?siri ba???layan kimi g�st?rs? d?, doxsan d?r?c? bucaq alt?nda o t?r?f, bu t?r?f?  ?yil?r?k faktlar?n d�zg�nl�y�n�n s?n?dl??dirlm?sind? m?n? mane�ilik etm?y? �al???rd?. Bu is? ondan ir?li g?lirdi ki, o, m?nim, Razmikin diqt?sind?n faydaland???m? bilirdi.\n M?n, Razmikin dedikl?rini v? �z�m�n a?kar etdiyim n�qsanlar? t?sdiql?m?k ���n t??kilat?n ba? m�hasibini yan?ma �a??rtd?rd?m. Tipik hay qad?n? idi. Y?ni, sif?tind?n, duru?undan, boy-buxunundan, ir?li �?xm?? qarn?ndan tutmu? �z�n�n q?pq?rm?z? r?ngin? v? i�inin x?lt?na kimi hay qad?n? oldu?u anla??l?rd?. \n Ba? m�hasib bu y�nd?msiz b?d?niyl? art?q qar??mda idi
 . �ox da yax?? dan??a bilm?diyi t�rk l?hc?sind?: - e?idiram sizi, -dey?r?k, ?ll?rini qarn?n?n �st�nd? c�tl?yib m?ni s�zd�.\n M?n, bir xeyli qar??mdak? ka??z-ku?uzu o �z, bu �z�n? �evir?r?k, n?d?n ba?layaca??m? g�t�r-qoy etdim:\n M?n, - dey? bil?rsinizmi, ?u?a ?�b?sinin r�bl�k mukafat?n?  n? s?b?bd?n k?smisinz?\n Ba? m�hasib dili-doda?? ?s?-?s?, -onlar at�otu gecikdirmi?dil?r, ona g�r?.\n -Ne�? g�n gecikdirmi?dil?r? \n - Be? g�n.\n  - Hadrut ?�b?si d? hesabat? gecikdirib, �z� d? on be? g�n, b?s nec? olur ki, onlar?n pul mukafatlar? �d?nilib, k?silm?yib?\n - Yox, gecikm?yib ha, kim bunlar? deyib siza, -dey?r?k, sa??mda oturmu? Razmik? t?r?f �t?ri d? olsa q?yqac? n?z?r yetirdi.\n M?n, onun Razmik? t?r?f  k?s? bax??larla d�nm?sind?n  bel? ba?a d�?d�m ki, o,? Razmik? i?ar? etdi ki, adam ki?i olar, s?n �z haylar?n? bir t�rk? satm?san v? s?n biqeyr?ts?n, el? olmasayd?n �ox vaxt s?ni t�rk? ox?atmazd?lar? fikirl??dim v? on
 un �z�nd?ki yalan?n q?zart?lar?n? saf-��r�k ed?-ed?: -onda buyurun h?r iki ?�b?nin hesabatlar?n? g?tirin m?n?, -dedim.  \n O, sif?ti q?zarm?? halda  otaqdan �?xd?, el? sif?tinin p�rtm�?� getm?mi? d? hesabatlar ?lind? geri qay?td?.  M?n �z�m� o yer? qoymadan, ?vv?l �t?ri d? olsa, ?u?a ?�b?sinin hesabat?n? o �z-b� �z�n? �evir?r?k gecikm? tarixinin oldu?u kimi qald??? v? ?l d?ym?d?n  t?sdiql?ndiyini m�?yy?n etdim. Hadrut ?�b?sinin hesabat?n?n tarixinin ba??na it oyunu a�m??d?lar. El? bil yek? bir v?l il? x?rmanda var-g?l etmi?dil?r, hesabat?n tarixi q?rm?z? q?l?ml? kobud formada pozulmu? v? �st�nd?n laz?m olan tarix yaz?lm??d?. \n M?n �z�m� ona t?r?f tutaraq: - b?s buna n? deyirsiniz? B?s bu d�z?li? deyil, n?di? �z�n�z d? bu doyda t??kilat?n ba? m�hasibisiniz. M�hasibatl?qda is? bel? d�z?li?l?r etibars?z hesab edilir. Nadir hallarda edil?n d�z?li?l?r d? m�hasibatl???n t?l?b etdiyi qanunauy?un �?r�iv?d? apar?l
 an d�z?li?l?r hesab edil? bil?r ki, o da, riyazi hesablamalar zaman? mexaniki v? nadir s?hvl?rin d�z?li?in? aiddir. Y?ni, nadir bir s?hv r?q?min �st�nd?n q?rm?z? q?l?ml? nazik x?tt �?km?kl?, onun k?nar?nda d�z?li? etdiyiniz dig?r r?q?mi yaz?rs?n?z v? bunun da s?b?bi ham? ���n b?lli olur. Ancaq b�t�n bunlar?n hesabatlar?n t?qdim olunma tarixl?rin? he� bir aidiyy?ti yodur. Siz is? hesabatda sanki kotanla yer ?kmisiniz.\n Xank?ndind? yerl???n t??kilat?n t?fti?�isi Razmikin ?l alt?ndan m?n? �t�rd�y� teleqram?n cavab?na t?krar?n bird? baxaraq:  -al?n bax?n v? yalan dan??may?n, bu da Hadrutdan ald???m?z teleqramd?, dey?r?k, ?limd?ki teleqram? ona uzatd?m.\n   Onun sif?tind?ki p�rt�k q?zart?lar h?ddini �oxdan a?m??d? v? art?q onun hay olmas?n? isbat etm?y? ehtiyac yox idi.\n Qula?? s?sd?ymi?, n?ydis?, t??kilat?n m�diri, ya?da v? g�rk?md? ba? m�hasibind?n geri qalmayan Qreta Nikolayevna  i�?ri girdi v? n? ba? verdiyini anlamaq ist?di. Onsuz da
  bu qad?n?n ?ovnist oldu?u m?n? �oxdan b?lliydi v? onu g�r?n kimi, yad?ma d?hlizd?ki ??r?f l�hv?si d�?d�. H?r halda ??r?f  l�vh?si ??r?fsiz v? insanlar? aldadan bir n?sn? kimi beynimd?n ke�s? d?, bu Xank?ndind? yerl???n ??r?f l�hv?sind? hay ?�b?l?rinin qabaqc?l hay i?�il?rind?n ba?qa, ?u?a ?�b?sind? i?l?y?n n? az?rbaycanl? v? n? d? rus mill?tind?n ibar?t bir n?f?rin d? olsa ??kili yox idi, ancaq orda da i?l?y?n haylar?n ??kili var idi. Bu n?sn?l?r x?rda  olsa da k�n�l buland?ran  n?sn?l?r idi. \n Hiss etdim ki, Qreta da h?rif deyil v? i�imd?n ke�?nl?ri ba?a d�?�r v? hiyl?g?r dilini i?? salmaq ist?yir, m?n buna imkan verm?d?n,  - Qreta Nikolayevna, nec? ola bil?r ki, ?u?a ?�b?sind? ancaq haylar yax?? i?l?mi? hesab edilir, az?rbaycanl?lar, ruslar yox, ?g?r yax?? i?l?y?n varsa, niy? bu ??r?f l�hv?sind? onlar da olmas?n?\n Qreta Nikolayevna bozarm?? ??kild?, -yox, yox, niy?ki var, m?n onlar? Bak?dak? ??r?f l�hv?si ���n g�nd?rmi??m.\n M?n o
 nun n? dediyini g�yd? tutaraq, - yax?? el?misiniz, siz g?r?k onlar?, el? bax bu Xank?ndind? yerl???n t??kilat?n ??r?f  l�hv?sind? d? yerl??dir?rdiniz, d�nya da??lmazd? ki.\n O, susma?? il? ?m?ll?rini �rt-basd?r etm?y? �al???rd?, ancaq gec idi. Eyni zamanda tez olma??n?n, yaxud gec olma??n?n onun ���n he� bir f?rqi yox idi. O, �z mill?t�i v? ?ovnist i?inin ustas? idi. Ancaq o da var idi ki, o, yax?? anlay?rd? ki, m?n onlar?n dabbaqda g�nl?rin? b?l?d idim v? yeri g?l?nd? sif?tl?rin? �?rpma?? bacarsam da,  onlar?n da yeri g?l?nd?, �zl?rin?  m?xsus y�k g�t�m?k qabiliyy?tl?ri var idi v? bu, bizi aldadaraq, ba??m?z?n alt?na yast?q qoymalar?yd?. El?,  Qreta da g�z�m�n �n�nd? bir anl??a y�kl� qat?r? xat?rlatd?  v? bu y�k�n alt?ndan xilas olma?a �al???rd?.\n H?l? onun iqtisad�?s? ?nessan? demir?m. Bir biqeyr?t az?rbaycanl?n?n pullar?n? Moskvan?n ?n bahal? mehmanxanalar?nda x?rcl?dib, kef �?k?-�?k?, az?rbaycanl?lar?n ?lehin? dan??ma?
 ?ndan da qalm?rd?. Dan???rd?, ��r?ymizi yeyb, suymuzu i�?-i�?, dan???rd?, o papa?? ?lind? qalm???n pullar?n? g�y? sovura-sovura...\n Hirs m?ni alt-�st etmi?di, ��nki ���m�zd?n biri, it oynadan idi, o biri d? bunlara i?l?y?n idi, ?traf?mda da haylar. M?nim d? bunlarla bel?, t?limata uy?un r?ftar?m?n qar??s?nda h?r n?sn? ola bil?rdi. Onu da hiss edirdim ki, onlar m?ni milli m?s?l?y? y�nn?ndirm?kl? q?z??d?rmaq v? n?y?s? nail olmaq ist?yirdil?r. Ancaq Xank?ndi t??kilat?n?n m�diri Qreta  �z hay dilind? n? x?rdalad?sa, ?nessa otaqdan �?x?b getdi. Bu minvalla, Qreta xahi? etm?k ist?yirdi ki, n�qsanlar? akt material?nda g�st?rm?yim. Salm??d? hay bicliyin?, ��nki H?s?n Qas?movi�d?n qorxmasa da, �?kinirdi.\n M?n art?q bu n�qsanlar? oldu?u kimi akt material?nda s?ralam??d?m.  \n\n VII\n M�zakir? H?s?n Qas?movi�in yan?nda idi. Idar? heyy?tinin m�avinl?ri, ?�b? r?isl?ri v? Qreta ba?da olmaqla,  Xank?ndind?n g?lmi? n�may?nd?l?r d? kollegiya 
 stolinun ?traf?nda �z yerl?rini tutmu?dular. Xank?ndind?n g?lmi? n�may?nd?l?rin i?tirak?ndan bilm?k olurdu ki, bu m�zakir? onlara g�r?di, ancaq m�avinl?rin v? ?�b? r?isl?rinin i?tirak? onu ?sasland?r?rd? ki, bu m�zakir? t?kc? yoxlama material?na g�r? deyildi, h?m d? Xank?ndind? f?aliyy?t g�st?r?n t??kilat?n b�t�vl�kd? i? f?aliyy?tin? y�n?ldilmi? bir m�zakir? idi. Buna baxmayaraq, ilk ?vv?l Moskvadan daxil olmu?  ?ikay?t ?riz?si ?sas?nda apar?lm?? yoxlama i?inin n?tic?l?ri dinl?nildi v? bu y�nd? ba?qa ?�b?l?rl? ba?l? dig?r n�qsanlar?n s?ralanma?? da, ona qar?? iradlar?n bildirilm?si d? m�zakir? obyektin? �evrilmi?di. M�zakir?y? geni? formada ba?lan?lmas?nna baxmayaraq, haya yax?n, ona yalmanmaq ist?y?n ?�b? r?isl?ri n�qsan?n birini q?sqana-q?sqana deyirdis?, dig?rinin �st�nd?n ke�m?y? �al???rd? v? yaxud da s?ralad?qlar? n�qsanlar? yenid?n yum?altma?a �al???rd?lar. Bir s�zl? m�dafi? m�vqeyind?n �?x?? edirdil?r.\n M?n h?l?
  �?x?? etm?mi?dim. �?x?? ed?nl?rin ?sil simalar?n? m�?yy?n etm?k ���n, g�z�m onlar da, fikirim is? �z�md? idi. G�z g�r?-g�r? h?r n?sn? �rt-basd?r edilm?y? y�n?lmi?di. G�z g�r?-g�r? bu boyda n�qsanlara yol verilmi?, bu boyda milli ?ovinist �ns�rl?rin t�?yan etdiyi t??kilat?n f?aliyy?ti, he� n? olmam?? kimi d?y?rl?ndirilm?y? y�n?lmi?di. Bax, onda milli m?s?l? bizim ���n xo?a g?lm?y?n hal kimi qar??lana bil?rdi. Ancaq milli m?s?l?ni az?ri t�rkl?rinin �z�n? g�zg�r?si qabardan v? Respublika t??kilat?n?n tabeliyind? olan Xank?ndind? yerl???n bu t??kilat?n ?ovnist v? mill?t�i i?�il?rinin ba? idar?d? �ox sayda t?r?darlar? da �ox sayda tap?l?rd?. Bu da guya biz? tarix?n verilmi? b???riliyimiz v? insanl???m?zdan ir?li g?lirdi. V? n?y? g�r? bu b???rilik, bu insanl?q t?kc? m?n? aid olmal?yd?, bu torpaqda ya?ayan ba?qa birisin? yox.  Bax bunu fikirl???n d? adam ancaq �z i�ind? dan??a bilirdi, �z i�ind? d�?�n? b
 ilirdi, onu da qorxa-qorxa, gizl?y?-gizl?y?, y?ni onda yerin qulaqlar? insan qulaqlar?ndan daha ??kl?nmi? g�r�n�rd�, onda onlar ���n yox, yaln?z  m?nl?r v? bizl?r ���n yasaq olan milli m?s?l? h?r add?mda bizim ���n t?hl�k? obyekti idi. V? tarix?n biz? qar?? y�n?lmi? bu qeyri insani h?r?k?tl?r, t?kc? biz? qar?? qanunil??irdi. Bir anl?q  Bak? v? Xank?ndi aras?ndak?  sovet imperiyas?n?n ?ovinist siyas?ti  m?ni rahat buraxmad???ndan, �z�md?n as?l? olmayaraq aya?a qald?rd? v? s�z�m oldu?unu bildirm?k ���n H?s?n Qas?movi�in �z�n? baxd?m.\n O, - h?, e?idir?m sizi, -dey?r?k, m?n? i?ar? etdi.\n- H?s?n Qas?movi�, -dedim, -burda �?x?? ed?n yolda?lar Xank?ndind? ba? \nvermi? olaylara s?thi yana?d?lar v?  onu m�dafiy? m�vqeyind? durdular. Bu da ondan ir?li g?lir ki, onlar?n Xank?ndin? gedi?l?ri v? ?m?lli-ba?l? qar??lanmalar?, onlar? m?cbur edir ki, bel? bir ciddi m?s?l?nin m�zakir?sin? bu y�nd? d?, qeyri leqal yana?s?nlar. ?slind? Xan
 k?ndind? yerli hay ?halisi il?, yerli v? k�kl� Az?rbaycan ?halisind?n ibar?t i?�il?rin aras?nda �ox b�y�k f?rqli yana?ma m�vcuddu. Bu da Xank?ndind? yerl???n bizim t??kilat?n hay mill?tind?n olan r?hb?rinin apard??? qeyri sa?lam v? milli siyas?tin n?tic?l?ridi. Burda akt material?nda qeyd edilmi? n�qsanlarla, aidiyy?ti ?�b?l?r tan?? olmas?na baxmayaraq, h?min n�qsanlar?n �st�nd?n s�kutla ke�dil?r. Qeyd etm?k laz?md?r ki, b�t�n bunlar ondan x?b?r verir ki, ?�b? r?isl?ri Qreta Nikolayevnan?n qar??s?nda sanki g�zl?ri k�lg?lidi, yaxud da ondan �?kinirl?r. Q?bul olunmu?, Az?rbaycan ssr-nin konstutisiyas?na ?sas?n Az?rbaycan dili �mumxalq dili olmas?na baxmayaraq, Xank?ndind? respublika t??kilat?n?n ist?r �z�nd? v? ist?rs? d? tabe�iliyind? olan t?kilatlar?n �nvan g�st?ricil?ri hay v? rus dill?rind? f?aliyy?t g�st?rir. Muxtar Respublikan?n tabeliyind? olan ?u?a ?�b?sin? ayr? se�gilik h�km s�r�r.  Az?rbaycan ?halisin?n s?x ya?a
 d??? dig?r rayonlar da bir n?f?r d? olsun  az?rbaycanl? kadr  bizim sistem? i?? g�t�r�lm�r. M?n respublikam?za aid Xank?ndi t??kilat?n? bizim t??kilatdan t?crid olmu? formada g�r�r?m. V? b�t�n bunlar?, bayaq �?x?? ed?n ??xsl?r g�r?-g�r?,  hans? x�susiyy?tin? g�r? bu xan?m?n i? f?aliyy?ti q?na?tb?x? hesab edil?r?k,  d?y?rl?ndirilm?lidi?  Ona g�r? d? bu i?? ciddi yana??lmas?n? xahi? edir?m.\n H?s?n Qas?movi� diqq?tl? m?n? qulaq asd? v? ?s?bil??mi? halda yerind?n dik qalxd?:\n- M?n, h?l? bir ne�? il bundan ?vv?l Xank?ndind?  olanda demi?dim\nki, Qreta Nikolayevna bel? i?l?m?k olmaz, bu ?ovnistliyinizd?n, bu mill?t�iliyinizd?n ?l �?kin. M?nim m�avinl?rim v? ?�b? r?isl?rim d? yoxlama materiallar? il? tan?? olmalar?na v? ba? ver?n n�qsanlar?n m�vcudlu?unu g�r?-g�r?, bu ciddi m?s?l?nin m�zakir?sind?n yan ke�m?y? �al???rlar. G�r�n�r Qreta xan?m onlar?n �z�n? h?mi?? yax?? bax?b, onlar da h?r n?sn?nin �st�n� malalay?b, �r
 t�b ke�m?k fikirind?dil?r. ?O, m?n? i?ar? ed?r?k?, - m?n yolda??n fikir v? t?klifl?riyl? tamam ??rik?m v? Qreta Nikolayevnaya ciddi t�hm?t elan edilsin, �� ay vaxt verilm?kl?, t?fti? ?�b?sin? h?val? edilsin ki, bu verilmi? vaxt m�dd?tind? d�n�? yarad?lmazsa, Qreta Nikolayevan?n v? aidiyy?ti i?�il?rin i?d? qal?b, qalmamas? m?s?l?si m�?yy?n edilsin.\n Az da olsa, H?s?n Qas?movi�in bu �?x???ndan v? Qreta Nikolayevnaya veril?n ??idd?tli t�hm?td?n?  bir q?d?r rahatlansam da,he� bu da m?ni qane etmirdi. Ancaq hiss etdim ki, c�zi d? olsa,  bu ya?ad???m torpa?a, bu v?t?n? balaca bir i? g�r? bilmi?dim. ��nki d�nyan?n imperiya vaxt? ?limd?n bundan ba?qa bir n?sn? g?lm?diyini yax?? bilirdim. V? bu bax?mdan i�imd?n a??r bir y�k g�t�r�lm�?d� v? sar???n z?minin tax?l d?rzisi q?d?r y�ng�ll??mi?dim.\n Qreta Nikolayevna tipik haya b?nz?r x?lt? i�ind?, suyu s�z�lm�? formada, t?k-t?nha k��?l?rin biriyl? add?mlay?rd?, g�t�rd�y�
 �� y�n, hay mill?tind?n olan ?�b? r?isl?rind?n birinin evin? t?r?f  ald??? y�n idi. Fikirl??dim ki, bu da bir hiyl?di, hay mill?tind?n olan ?�b? r?isi �nc?d?n gedib v? bilir ki, Qreta onun evind? �ox oldu?undan evi yax??  tan?y?r v? �z� g?lib �?xacaq. �l�� c?zas?n? ?hat? ed?n ?mr? birinci d?rk?nar? m?n qoydu?umdan  �z�m� rahat hiss edirdim.  \n\n VIII\n\n T?fti? ?�b?sinin r?isini haylar ?baqaja? qoymu?dular. Guya  �z�n� apara bilmir, ?alka?d?? v? s. v? i.\n El? bu s?b?bd?n d? t?fti? ?�b?sin? t?z? r?is g?lmi?di, q?r?m?ndan bacar?ql? adama ox?ay?rd?. C�nki h?r i?in yerini-yata??n? bilirdi v? burda alver etm?k, bazarl?q yapmaq, r�?v?t d? yox deyilddi. H?tda pamb?qla ba? k?sm?k d? yox deyildi.\n Yuxar?larda  q?bul edilmi? q?rarlar?n n?tic?sind? pul d�vriyy?si yax??la?m??d?, t?kc? qal?rd? ki, onun �hd?sind?n ?layiqinc?? g?l?s?n, o da bu t?z? r?isin ?lind? ?semi�ka? �?tdamaq kimi bir n?sn?ydi.\n M?n aran rayonlar?n?n birind? ezamiyy?d? i
 dim. ?�b?d?,  Bak?dan kadrlar ?�b?sind?n z?ng edib m?ni axtard?qlar?n? bildirdil?r.    �mumiyy?tl?, ke�mi? ittifaq?n telefon ??b?k?si, ?sas?n sifari?li xarakter da??y?rd?. Avtomat telefon stansiyas? vasit?siyl? kadrlar ?�b?sin? z?ng etdim, salam-k?lamdan sonra, sabah tezd?n v?  t?cili Bak?da olma??m t?l?b edilirdi. S?h?r tezd?n, y?ni i? ba?layan vaxt? �z�m� kadrlar ?�b?sinin ota??na sald?m: - n? m?s?l?di, - dey?, narahat formada maraqland?m.\n- Biz d? bilmirik, sizi H?s?n Qas?movi� ota??nda g�zl?yir, -dey? kadrlar\n?�b?sinin r?isi bildirdi.\n S?drin yan?nda m?nd?n ba?qa t?fti? ?�b?sinin t?z? r?isi d? var idi. O, s?drin qar??s?nda �z�n? m?xsus g�rk?md? sakit v? dinm?z oturmu?du. M?n d? s?drin t?klifin? r??m?n stullar?n birind? ?yl??ib, n? s?b?b? �a??r?lma??m?n n?tic?sini g�zl?dim. Ancaq narahatl??a d?ym?diyini hiss etmi?dim v? bilirdim ki, h?r halda yax?? i? ���n �a??r?lm??d?m. El? olmas?n?, ondan hiss etdim ki, s?dr ?vv?l m?nim �z�m? v
 ? sonra da t?z? r?isin �z�n? xo? ehyamla baxd? v? t?fti? ?�b?sinin r?isin? yenid?n d�n?r?k s�z? ba?lad?:\n- Bu yolda?? y?qin ki, tan?y?rsan, i?l?diyi m�dd?td? �z�n� bacar?q?, \nt?cr�b?li v? savadl? bir kadr kimi g�st?rib, ?lav?d?, qeyd edim ki, sovet ittifaq? kommunist partiyas?n?n �zv�d�.\n Fikirl??dim ki, ax?, s?drin hesabat?n?n m?n? n? aidiyy?ti, ancaq fikirim-zikirim ondayd?. O, aram-aram, fasil? verm?d?n dan???rd?:\n- H?, m?n, onun kimliyi bar?d? ?trafl? m?lumat alm??am, v? g�r�r?m ki, \nl?yaq?tli v? bacar?ql? kadrd?r.\n Bu yerd? m?nim yad?ma d�?d� ki, t?fti? ?�b?sinin t?z? r?isini yax?ndan tan?y?ram, h?tda k�hn? r?isimin i?tirak? il? onunla nahar da etmi?ik. Eyni zamanda k�hn? r?isim t?r?find?n  bu adam?n yazd???, hans?sa, bir ?lyazmas?n?n eyb?c?r d�z?li?l?ri d?, bu d�z?li?l?rd?n �t�r� bir ne�? d?f?, onun yan?na g?lm?si d? yad?ma d�?d�.\n Bu fikirl?rin duman?nda aran? da?a, da?? arana da??d???m anda, H?s?n Qas?movi� y
 �ks?k v? r?hb?r tonunda m?n? t?r?f d�n?r?k, nitqini davam etdirdi:\n- M?n, t?fti? ?�b?sinin t?z? r?isi il? s?n g?lm?mi?d?n ?vv?l, s?nin bar?nd? \nbir xeyli fikir m�badil?si emi?ik v? bu t?klif d? m?nim ??xsi t?klifim olub. Q?raq bucaqdan t?klifl?r �ox idi, m?n is? bir t?klifin �z?rind? dayanas? oldum v? t?fti?in t?z? r?isi il? d? bu bar?d? raz?l??a g?l?si olduq. Bu t?klifin �z?rind? dayanma??m is? el? bel? deyil, bir s?ra i?l?rd? s?ni s?naqdan ke�irmi??m v? his etmi??m ki, t?miz adams?n?z, sistemin t?snifini he� kim? vermirsiniz v? sistemin m�dafi?sinin �hd?sind?n layiqinc? g?l? bilirsiniz, tamahs?z adams?n?z, insanlara qar?? davran???n?z m�sb?tdi, ��nki biz ?hali il? i?l?yirik, onlara qar?? qay?? bizim ?sas silah?m?z olmal?d?r.    B�t�n bunlar? n?z?r? al?b, sizi t?fti? ?�b?sinin r?isin? m�avin t?yin etm?k ist?yir?m, t?klifin n?dir.\n M?nim bu m?s?l?d? t?r?dd�t etm?yim, yaxud etm?m?yim n? ver?rdi ki, ax?, fikirl??dim v?, -m?sl?h?t sizindir, -dey
 ?r?k, yeni g?lmi? r?isin �z�n? baxd?m.\n O, daxili telefonu g�t�r�b kadrlar ?�b?sinin r?isin?, - Niyazovun ??xsi i?ini g?tirin, g?lin yan?ma, -dey?r?k telfonun d?st?yini g�t�rd�y� yer? d? qoydu.\n Kadrlar ?�b?sinin r?isi, ucaboylu, sar???n, sar???n da b??lar? olan orta ya?l? ki?i ?lind? d? m?nim ??xsi i?im i�?ri girdi v? h?min s?n?dl?ri s?drin stolunun �st�n? qoydu. \nS?dr m?nim ??xsi i?imi diqq?tl? v?r?ql?yib, kadrlar ?�b?sinin r?isin? i?ar?yl?, m?ni g�st?r?rk, -bu g�nd?n etibar?n bu yolda?? t?fti? ?�b?sin? r?is m�avini t?yin edir?m v? ?mrl??dirib t?sdiq ���n t?qdim edin, -dedi.\n Sovet t?hl�k?sizlik sisteminin k??fiyyat�?lar?na ox?ayan bu sar???n ki?i, - ba? �st? H?s?n Qas?movi�, -deyib, otaqdan �?xd?.\n M?n? g�st?rdiyi etimada g�r? H?s?n Qas?movi�? t???kk�r�m� bildirib, otaqdan �?xd?m v? t?fti? ?�b?sin? qay?d?b, stulumda ?yl??dim. I? yolda?lar?m bir az ?�bh?li-?�bh?li m?n? baxd?lar, m?n is? he� bir n?sn? 
 olmam?? kimi, t?fti?i i?inin apar?lmas?na aid t?limat qaydalar?n? v?r?ql?yib oxusam da, fikirim ayr? yerd? idi. Bir az ke�m?di ki, t?fti? ?�b?sinin t?z? t?yin edilmi? r?isi d? ota?a daxil oldu, c?ld v? ??stli ad?mlarla ir?lil?yib �z yerind? oturdu. Bel? fikirl??m?k olard? ki, v?zif? adamlar?n?n ham?s?nda bu x?st?lik ?n?n?vi xarakter da??y?rd?. B?z?n �zl?rini ?trafdan t?crid olmu? kimi v? yaxud t?k?bb�rl� g�st?r?n, �z�n? s??mayan, h?tda bel? al�aq da?lar? yaratm?? kimi, t?sir ba???layan r?isl?r? d? rast g?lm?k olard?. Hiss etm?k yerin? d�?�rd� ki, bu t?z? g?lmi? r?isin d? iddialar? balaca deyildi, �z�nd?n b�y�k idi. Bunu onun davran???na ?sas?n d? m�?yy?n etm?k olard?.\n T?z? r?is, ani bir s�kutdan sonra �z�n� t?fti? ?�b?sinin t?fti?�il?rin? tutub,\n - bilirsiniz, s?drimiz H?s?n Qas?movi� m?ni yan?na �a??rm??d?, bir xeyli s�hb?t etdik v?  m�avin m?s?l?sind? h?r ikimizin fikirl?rimiz v? t?kilifl?rimiz �st-�st? d�?d�, art?
 q bu g�nd?n etibar?n yolda? Niyazov  m?nim m�avinimdir, t?limat qaydalar?na aid n? sual?n?z olsa onunla h?ll ed? bil?rsiniz, i? bar?d? m?s?l?l?ri m?niml? b�l�?d�r?c?ksiniz.\n Ir?li �?kilm?yim? g�r? i�imd? sevinc v? q�rur hissi ke�irs?m d?, h?lli m?nmlik olan m?s?l?l?rin h?ll edilm?sind? m�st?qil olmama??m?n a?r?lar?n? indid?n anlay?rd?m.   Onu da yax?? anlay?rd?m ki, bu ir?li �?kilm?yim d? r?mzi xarakter da??y?rd?. ��nki ist?r s?dr v? ist?rs? d? ?�b? r?isi yax?? bilirdi ki, kimi hans? v?zif?y? t?yin etm?k  g?r?klidir. Demaqoq olmas?n, ortada olan? orda-burda dan??mas?n, tamahkir olmas?n, ortadak? g?lirl?rd? g�z� olmas?n, dindir?nd? dinsin, dindirm?y?nd? d? lal-dinm?z yerind? otursun. Dem?li m?nim robotdu?um daha m?sl?h?tliymi? ki, m?ni d? ona m�avin t?yin etmi?dil?r. ?g?r ke�mi? r?isin yerin? ?bacar?ql?? v? ?qurd kimi? da??d?c? bir adam g?tirmi?dil?rs?, onda ona m�avin el? adam olmal?yd? ki, onun i?in? burnunu soxmas?n, tikans?z olsun, onun ci
 ddi �at??mamazl?qlar? olsa da bel?, g�z�n� yumsun. Yoxsa ke�mi? m�avin, M�rs?l kimi, burnunu tez-tez, n?in ki, r?isin i?-g�c�n? soxurdu, h?tda ne�? i? yolda??n? orqan i?�il?rinin ?li il? l?k?l?yib s?radan  �?xartm??d?.  N?di-n?di, yoxlama zaman? t?fti?�i, filan s?n?dd? bu n�qsan? g�r? bil?rdi v? yaxud g�rm?y? d? bil?rdi v? h?r n?sn?y? ehtimal g�z�yl? baxan t?fti? ?�b?sinin r?is m�avini M�rs?l, bu minvalla �oxlar?n?n dal?ndan d?ydirmi?di, �z� d? h?r?nin belind? bir iz buraxa-buraxa. Bax onda m?nim yad?ma bank S�hb?tin qarda?? Abdulla m�?llim d�?m�?d�. Ehtimal n?z?riyy?si m�?llimi. O, adi misallarla bu f?nnin maraql? t?r?fl?rinin d? ehtimal oldu?unu izah ed?-ed?, bizi ehtimallara k�kl?y?rdi. Ancaq Abdulla m�?llimin ehtimal?ndan  f?rqli olaraq M�rs?lin ehtimal? qorxu hissind?n yaranm??  ehtimallardan biriydi, y?ni ?g?r orqan i?�il?rind?n biri onun �st�n? q??q?rsayd? ki, bu s?n?d d? bu n�qsan? g�rm?k olard?, ya 
 yox, o da a??na-qaras?na fikir verm?d?n dey?c?kdi ki, ?b?li yolda? hakim g�rm?kd? olar, g�rm?m?k d??. Bu da o s?n?dl?ri ara?d?ran v? M�rs?ll? birg? f?aliyy?t g�st?r?n i?�il?r? baha ba?a g?l?c?kdi. Bax, yolda?l?qda M�rs?l, yolda?l??? yox, �z�n�n can qorxusunu m�dafi? ed?c?kdi. V? onun bu can qorxusu is? onlara baha ba?a g?l?c?kdi, ?m?k kitab�alr? korlanacaqd?, i? yerl?rini itir?c?kdi, t?z? i? yerl?rin? g�t�r�lm?y?c?kdi v? s.\n Fikir m?ni aranl?-da?l? g�t�rd�y�nd?n i? yolda?lar?m?n nahar fasil?sin? getm?l?rini v? nahar fasil?sinin olma??n? bel? unutmu?dum.  Hiss etdiyim o olmu?du ki, otaqda r?isl? m?n ba?a-ba?a qalm??d?m. B?lk? d?  r?is dem?s?ydi ki, nahar fasil?sin? niy? getm?mis?n, el? ?limd? olan bu t?limat b�t�n g�n�m� b�sb�t�n ?limd?n alacaqd?. O, f�rs?td?n istifad? edib, m?niml? i?in gedi?at?n?, rayon ?�b?l?rinin hans? yax??, hans? pis i?l?m?sini v? s. ayd?nla?d?rmaq ist?di.\n O, divardak? mar?urut x?rit?sin? yax?nla?d?, sa
 ? ?lind?ki �ubu?u x?rit?nin �st�nd? g?zdirdi. Bir-iki ya?l? zonan? m?n? g�st?rdi v? soru?du: -bu zonalar nec?dir, olmusanm? bu zonalarda, -dedi.\n Onun n? dem?k ist?diyini alm??d?m, el? ona g�r? d?, -yox, -dedim.\n O, ?lind?ki �ubu?u x?rit?d? g?zdir?-g?zdir?, - e?itmi??m ki, bu zonalar yax?? pul ver?n zonalard?. S?ni g�nd?r?c?m bu zonalara, ��d?n birin?.\n M?n o yax?? zonalar? ba?a d�?s?m d? he� birind? olmam??d?m v? el? bu s?b?bd?n d? bu ��d?n biri m?s?l?si m?ni ya bil?r?kd?n, ya da bilm?y?r?kd?n almad? v? soru?dum, - r?is bu ��d?n birini ba?a d�?m?dim, bu n? dey?n n?sn?dir?\n O, bic-bic g�l�ms?di v? k��?y? baxan p?nc?r?y? t?r?f ba??n? qovzaya-qovzaya, - s?n indiy? kimi i?l?yirs?n, bilmirs?n ��d?n biri n?di? Y?ni s?ni ?n yax?? zonalara g�nd?r?c?m, ald???n r�?v?tin ikisi s?nin olacaq, biri is? m?nim.\n M?n fikirl??m?d?n k?s? cavab verdim, - r?is m?ni ??n pis? rayonlara g�nd?r, - dedim, ��nki ??n pis? rayonlar, kas?b, y?ni g?liri 
 a?a?? olan v? bunun m�qabilind? d? pul verm?y? imkanlar? olmayan rayonlar idi.\n O,  bic-bic �z�m? baxaraq, -m?nim ���n f?rqi yodu, m?n s?nin ���n deyirdim. Fikirl??dim ki, s?nin d? ail?n, u?a??n var, -dedi.\n M?n k?s? cavab verdim, - a?r?maz ba??m? niy? a?r?d?m ki, -dey?, fikirimi q?til??dirdim v? unutmu? oldu?um nahar fasil?sin? yolland?m.\n El? h?min g�nd?n d? m?nim taleyim? bir a??rl?q d�?d�, ancaq m?n buna pe?man deyildim. T?kc? pe?man�?l?q ondan ibar?t idi ki, r?is art?q m?ni incitm?y? ba?lam??d?. Bu incitm?yin s?b?bl?ri m�xt?lifdi, adam-adam? ist?m?s? hans? yolla ist?s? bezdir? bil?r, can?n? bo?az?na y??a bil?r. O, art?q bu minvalla da ba?lam??d? m?ni tez-tez, dabanbasd? ezamiyy?l?r? g�nd?rm?y?, aran v? da? rayonlar?na m�vs�md?n as?l? olaraq, qrafik tutulmas?na baxmayaraq, q???n q?? vaxt? da? rayonlar?na, istinin isti vaxt? is? aran rayonlar?na g�nd?rm?y?. O q?d?r tez-tez g�nd?rirdi ki, bir g�n qap?n? d�y?nd? b�y�k q?z?m qap?n?n g
 �zl�y�nd?n bax?b anas?na demi?di ki,? hans?sa ?mi? var qap?n?n �n�nd?, anas? da qap?n? a�?b, -q?z?m ?mi deyil, atand?, -demi?di.\n O q?d?r bezmi?dim, �al???rd?m ki, i? yerimi d?yi?im, r?isim bu i?d? d? m?n? mane�ilik edirdi. Bunlar? t?fti? ?�b?sin? daxil olan telefon z?ngl?rind?n hiss edirdim, ��nki otaqda ??h?r telefonu bir d?n? idi o da r?isin stolunun �st�nd? idi. B�t�n bunlar? telefon z?ngl?rin? ?ifahi v? m?xvi veril?n m?lumatlardan hiss edirdim. Eyni zamanda bunu m?n, t?z? i? yerl?rind? ilk ?vv?l, m?ni iki ?lli qar??lay?b, n�vb?ti  g�n is?? m�mk�n olmad?? v? yaxud ?siz? aid bizd? i? olmad????n? bildirirdil?r. Ayd?n olurdu ki, r?is m?ni ?m?lli-ba?l? m?nfi xarakteristikas? il? h?r n?sn?d?n uzaqla?d?r?rm??. \n\n IX\n\nYen? d?, aran b�lg?l?rind?n birind? ezamiyy?td? idim. Bak?dan z?ng etmi?dil?r. Yen? H?s?n Qas?movi�in m?ni axtard???n? x?b?rdar etdil?r. Ad?ti �zr?, ezamiyy?d?n qay?td???m g�n�n s?h?ri i?? �?xd?m. \nBirba?a s?drin yan
 ?na getdim. H?s?n Qas?movi� �z�n� m?n? tutaraq: - bilirsiniz,? t??kilatlarla i?l?y?n ?�b??nin r?isini i?d?n azad etdim, i?l?y?n deyildi. ?�b?nin i?i getmirdi, m�asir texnoloji prosesl?rl? ayaqla?a bilmirdi. Nazirlik v? ba? idar?l?rl?, el?c? d? yerl?rd? aidiyy?ti struktur b�lm?l?ri il? apar?lan i?l?r yar?tmazd?. ?limizd? r?hb?r tuta bildiyimiz bir �ox s?n?dl?r olmas?na baxmayaraq, yararlana bilm?dil?r v? bu imkanlardan istifad? ed? bilm?dil?r. Halbu ki, Az?rbaycan m?rk?zi komit?sinin,  bizim t??kilat?n inki?af?na y�n?ldilmi?, b�ronun q?rar? var. Ssri nazirl?r sovetinin, Az?rbaycan nazirl?r sovetinin, ssri d�vl?t bank?n?n, el?c? d? bizim Moskvadak? ba? idar?mizl? ssrinin b�t�n nazirlikl?ri v? ba? idar?l?ri il? birg? g�st?ri?l?ri m�vcuddur, �ox t??ss�f ki, ad? �?kil?n ?�b? yerind? say?r v? bu m?s?l?l?rin h?llini t??kil ed? bilmirl?r. Bizim m?qs?dimiz ?hali aras?nda  m?d?ni xidm?t sferas?n? genil?ndirm?k, y�ks?k s?viyy?d? xidm?t i?i yaratmaqd?r.
  M?n ne�? g�nd�r ki, bu bar?d? g�t�r-qoy etmi??m. D�zd�, bir ne�? n?f?r �st�m? minn?t�i sald?lar, yuxar?lardan z?ng etdirdil?r, o ?�b? onlar?n yeri deyil, o ?�b?d?, g?r?k ?lini �?rmalay?b, ?m?lli-ba?l? i?l?y?s?n. Odur ki, o ?�b?nin aya?a qald?r?lmas?n? s?nin i? qabiliyy?tinl?, bacar???nla v? savad?nla ba?l? oldu?unu fikirl??dim. H?r  hans? bir �?tinliyin  olsa, o m?s?l?l?rin h?llind? siz? k�m?k  edil?c?kdir. ??xs?n m?n �z�m o ?�b?nin i?ini n?zar?t? g�t�rm�??m, h?r hans? nazirlik v? yaxud yuxar? dair?d?n mane�ilik olarsa, m?ni m?lumatland?rarsan v? vaxt?nda onlar?n qar??s?n? alar?q. Odur ki, s?n?dl?rini haz?rla Moskvaya getm?lis?n, ��nki ?�b? r?isl?ri Moskvan?n ?tat vahidl?ridi. ?�b? r?isl?rinin i?? g�t�r�lm?si onlar?n q?bul etdiyi q?rara ?sas?n h?yata ke�irilir. M?s?l? art?q onlarla raz?la?d?r?l?b v? s?nin h?min ?�b?y? r?is t?yin edilm?yini  m?sl?h?t g�rm�??m. \n -H?s?n Qas?movi? h?r halda o ?�b?d? insititutdan 
 t?yinatla g?lmi? bacar?ql?, savadl? v? perspektivli g?nc kadrlar da olmal?d? ax?, -dedim.\n - El? onlar?n i?i m?ni qane ets?ydi s?h?rd?n s?ninl? bir quranl?q dan???q aparmazd?m. S?h?r getm?lis?n. Yax?nla? kadrlar ?�b?sin?, hans? s?n?d laz?msa,  haz?r el? v? Bak?-Moskva  reysin?, s?h?r u�u?u ���n indid?n bilet al. Ssri �zr? ba? idar?nin s?dri x?st?dir, i?? h?l?lik �?xm?r, s?ni birinci m�avin q?bul ed?c?k, �z�n� savads?z g�st?rm?, s?n? veril?n suallar i?imizin sah?l?rin? aid olacaq, t?l?sm?, ?vv?lc?d?n fikirl??, sonra cavabland?r.  \n T???kk�r edib otaqdan �?xd?m. Kadrlar ?�b?sin? yax?nla?d?m, qap?n? a�?rd?m ki, ke�mi? ittifaq?n t?hl�k?sizilik sisteminin polkovnikin? ox?ayan kadrlar �zr? r?is m?niml? qar??la?d? v? kobud sur?td?, -s?n d�n?nd?n burda olmal?yd?n, niy? gecikmisiniz, - dey? m?ni m?z?mm?tl?di.\n I?? ke�m?dimmi, �z-�z�m? gleyl?n?r?k, -ax?, siz m?n? d�n?n x?b?r etmisiniz, m?n d? d�n?n qay?td?m v? bu g�nd? i?d? oldum, h?m d
 ? el? ezamiyy? m�d?ti d? d�n?n qurtard?. El? s?drin yan?ndan �?x?ram, bilm?liydim ki, s?dr m?ni niy? axtar?b, bel? ba?a d�?m�?d�m ki, m?s?l? yoxlamayla, n? is? bir �at??mayan bir n?sn?yl? ba?l?d?. Ayd?n oldu ki, o, m?ni ba?qa bir m?s?l? il? yan?na �a??r?bd?.\n O, - yax??-yax??, ke� otur, g?lir?m, - dey?r?k, otaqdan �?xd?.\n Ota?a daxil oldum v? onun m�avini zarafat�?l oldu?undan bir az m?z? q?ld?q. M�avin Moskvada oldu?undan, bizim ordak? ba? idar?mizd? tan??? bir xan?mla s?rg�z??tl?rind?n dan??d?  v? salam?n? ona �ad?rma??m? xahi? etdi. M?n d? eyni ?hval-ruhiyy?d? onu arxay?n sald?m. Kadrlar?n ?�b? r?isi is? ?lind? apard??? qovluqla da geri qay?td?. M?n laz?m? s?n?dl?ri doldurdum. T?kc? partiya t??kilat?n?n xasiyy?nam?si qalm??d?. O da yaz?lm??d?, ?�b? r?isi doda??n?n alt?nda m?z?ldaya-m?z?ldya m?n? uzadaraq: - t?cili imzalat, g?tir m�h�r vuraq,- dedi.\n Otaqdan �?xd?m, partiya t??kilat katibinin ota??na daxil olan da, tan?d???m biriyl? hans?
 sa m?s?l?ni m�zakir? edirdil?r, m?ni g�r?n kimi, - i?in t?cilidi? - dey?, m?n? diqq?t k?sdi.\n- B?li, dedim, partiya t??kilat?n?n xasiyy?tnam?si yaz?l?b, ancaq \nimzalanmay?b, t?cili imzalanmal?d? v? m�h�r vurulmal?d?, s?h?r Moskvaya u�mal?yam, t?sdiq ���n.  \n O, - c?ld ol, c?ld ol, i?imiz �oxdu, -deyib, xasiyy?tnam?ni �z d?sti-x?ttil? imzalad?\n M?n otaqdan �?xd?m v? t?l?sik, yenid?n kadrlar ?�b?sin? yax?nla?d?m, s?n?d m�h�rl?ndi, indi yubanmadan  hava yollar?n?n bilet kassas?na t?l?sm?yim qal?rd?, h?l? bazara da d?ym?liydim. Laz?m? s?n?dl?r qoltu?umda, birba?a bilet kassas?na yolland?m, h?l? g�norta�st�n? xeyli zaman var idi. �st?lik bazara da vaxt qal?rd?, ?libo? at?la-at?la getm?yin d? l?zz?ti yox idi. Bilirdim ki, moskval?lar Az?rbaycan?n nar?n? v? konyak?n? �ox sevirl?r. \n Art?q bilet  kassas?nda idim v? xo?b?xtlikd?n adam az idi,  bilet d? var idi. Bilet ald?m v? s?n?dl?rin yan?na qoydum, fikirl??dim ki, ev? gets?m gec olacaq, yax??s? b
 udur birba?a bazara gedim. Taksil?r d? m?ni g�rm�?d�, biri d? g�z? d?ymirdi, yan?mda k�hn?-salxaq, nuh ?yyam?ndan qalma bir moskvi� dayand?, mindim v? s�r�c�d?n xahi? etdim ki, t?z? bazara s�rs�n v? be?-on d?qiq? d? m?ni g�zl?sin. Onsuz da bazardan nardan ba?qa bir n?sn? almal? deyildim. Ald???m be?-on kilo narla tez-t?l?sik ma??na qay?td?m. Qal?rd? konyak m?s?l?si, konyak alsayd?m, pul �?tinliyim olacaqd?, almasayd?m da,  nar h?diyy?l?rimi konyaks?z verm?yim �?tin olacaqd?. F�rs?td?n istifad? edib, ??rab zavodunun yan?nda f?aliyy?t g�st?r?n ?�b?nin m�dirin? z?ng etdim. H?r�?nd ki, onun m?nd?n xo?u g?lmirdi. ��nki i?d? buraxd??? n�qsanlara g�r? hay mill?tind?n v? apar?c? i?�il?rind?n birini i?d?n �?xartd?rmaq ist?yirdim, el? h?min ?�b?nin m�dirinin adam? oldu?una g�r?, yalvar-yaxardan sonra t�hm?tl? kifay?tl?nmi?dik.\n Z?ng ed?r?k, ?�b? m�dirini ad?yla �a??rd?m, telofonun o ba??ndan, -b?li, b?li e�idir?m, sizi, - dey?
 r?k, m?ni dinl?m?y? �al??d?, �z�m� t?qdim ed?r?k, - siz? i?im d�?�b, -dedim.\n Sakit-sakit, asta v? aramla, - e?idir?m, - dedi.\n -m?n? t?cili 20 ?�?? t?miz zavod m?hsulu olan konyak laz?md?r,- dedim.\n Telefonun o biri ba??ndan, -n? vaxta?\n -ax?am �st�n? kimi, ��nki s?h?r reysi il? Moskvaya u�mal?yam.\n -bir g�n ?vv?l dey?ydin, - dey?, m?z?ldand?.\n Ona  izah etm?y? �al??d?m, - m?n �z�m d? he� bilm?mi??m, ezamiyy?td? idim, d�n?n x?b?r edibl?r, bu g�n d? g?lib s?n?dl?ri yan-y�r? etmi??m, ma?azaya getm?y? vaxt olsa da, onlar?n konyaklar?na bel ba?laya bilmir?m, h?r-halda onlar sizin xidm?tinizd?dirl?r. Bilir?m ki, siz? keyfiyy?tli m?hsul ver?rl?r. Bizim Moskvadak? ba? idar?mizin bir ne�? i?�isin? h?diyy? etm?k ist?dim, -dey?, onu ba?a salma?a �al??d?m.\nBir saatdan sonra z?ng etm?yimi bidirdi. Bir saat �ox gec g?lib ke�di, ??rin qar??mas?na az qal?rd?. Bu, m?nd?n as?l? bir n?sn? deyildi, zaman dar?xd?r?c? idi.\nTelefon d?sd?yini g�
 t�r�b, bir d? z?ng etdim, katib?siydi, be? d?qiq?d?n sonra z?ng etm?yimi bildirdi. Be? d?qiq?d?n sonra yenid?n z?ng etdim, indi d? dedil?r ki, m�dir harasa �?x?b. Bir az h�vs?l?mi basd?m. Ba?qa n? etm?liydim ki, o q?d?r cibimd? pul olsayd?, m?n bel? har?n adamlara a??z a�ard?mm?, dey? �z-�z�m? var-g?l etdim. Yenid?n z?ng etm?kd?n ba?qa �ar?m qalm?rd?, telefonun d?sd?yini g�t�rd�m, k�n�ls�z-k�n�ls�z v? a??r-a??r n�mr?ni nec? y??m??d?msa, he� �z�md? hiss etm?mi?dim. Bu d?f? katib? m?ni �ox g�zl?tm?di, -indi qo?uram, dan???n,\n -dedi.\n Telefonun o biri ba??ndan m�dirin s?si g?ldi, - dediyin n?sn?l?r haz?rd?, hara g�nd?rim, -dedi.\n -i?d? g�zl?yir?m, bura g�nd?rs?n yax?? olar, el? buradan da ev? g�t�rr?m v? qavqarram, - dedim.\n Bu i? d? bel? d�z?ldiyind?n ba?qa bir i?im qalmam??d?, ax?amdan h?r n?sn?ni d�zd�m qo?dum v? s?brsizlikl? s?h?rin a�?lmas?n? g�zl?dim. Bilirdim ki, tezd?n bir q?d?r taksi �at??mamazl??? o
 lur, bir az ona g�r? narahat idim. \n S?h?r o ba?dan durmu?dum. �z�m� tara? etdim, b?n�v??yi k�yn?yimin yaxas?na ona uy?un qalstuk tapd?m, qaraya �alan kostumumu geyindim, ?limd?ki, �ox da iri olmayan y�k�ml? taksi dayanaca??na add?mlad?m. Taksi dayanaca??n?n da ad? var idi, ?ks?r vaxtlarda orda ara taksil?ri dayan?rd?, indi he� onlar da yox idi. G�zl?y?si oldum, ancaq hava liman?na gecikm?yim? g�r? dar?x?rd?m. Anca el? bil hiss edir?mmi? kimi, hardansa, k�hn? bir g�y volqa peyda oldu, qaz 24-l?r moda mins? d?, bu k�hn? qaz 21-l?r �z yerini h?l? bu t?z? ma??nlara verm?mi?di v? tank kimi, ora-bura ?�t�y�rd�. Ma??n yan?mda durma??yla, sor?u-suala tutmadan minm?yim bir oldu. ��nki �nvan? ?vv?l dem?kd?n �?kindim, fikirl??dim ki, bird?n uzaq yola getm?kd?n ehtiyat ed?r. S�r�c� o q?d?r d? pis adama ox?am?rd?, min?nd?n sonra ged?c?yim �nvan? bildirdim. \n Hava liman?na �atan kimi, tez qeydiyyat m?s?l?sini h?ll etdim, y�kl?rimin 
 d? baqajda yox, ?limd? aparaca??m? bildirdim v?  m�?yy?n s?b?b? g�r? ?limd? aparma??n icaz?sini d? ald?m. \n A??r y�kl? birlikd? s?rni?inl?r? qar??m??d?m, g�zl?m? zal?nda g�zl?m?yimiz uzun �?km?di, art?q daxili avtobusa minik ba?lam??d?. Bir t?h?r avtobusda �z yerimi tutdum v? Moskvaya y�n alas? u�a??n yan?nda idik. Biletim ikinci salona idi, ?vv?l ?limd?ki y�k� oturaca??n alt?nda, �?tinlikl? d? olsa yerl??dirdim. B?l?d�inin x?b?rdarl??? g?lm?d?n ehtiyat k?m?rini belim? vuraraq, yerimi m�hk?ml?ndirdim. ?slind? k?m?rin uclar?n? bir-birinin �st�n? qoymu?dum, �z�m� �idarlamaqdan z?hl?m gedirdi v? bel?lik?, sanki b?l?d�inin g�z�nd?n p?rd? asm??d?m.\n U�a??n m�h?rrikinin s?si g?ls? d?, h?l? yerd?ydik, bu, u�a??n s?maya qalxmam??dan ?vv?l yar?m saatl?q haz?rl?q v? nasas?zl???n?n saf-��r�k edilm?sin? ayr?lm?? zaman vaxt?yd?. Laz?m olan vaxt m�d?tind?n sonra, mavi r?ngd? pilot paltar? geyinmi? b?l?d�i peyda oldu, mar?urutu, �at
 as? zaman vaxt?n? elan etdi, ancaq k?m?rl?rin bel? b?rkidilm?sini d? unutmad?. U�aqdak? yerim xo?b?xtlikd?n, p?nc?r?nin yan?na d�?m�?d�, getdikc? yerd?n uzaqla??rd?q, s?man?n yuxar? qat? buludlarlanan ?h?t? edilmi?di. Art?q s?mayla buludlar?n aras?ndayd?q, buludlardan a?a?? he� n? g�r�nm�rd�. Sanki d?niz dal?alar? �st�nd? g?mimiz �z qanadlar?n? g?rib arxay?n v? y�ks?k s�r?tl? ir?lil?yirdi. Yan?mdak? oturacaqda oturan xarici adam t?siri ba???lad?, ?vv?l koreyal? kimi t?s?vv�r etdim, ancaq yapon oldu?u m?lum oldu. T?sad�f�, �z�ml? g�t�rd�y�m Yapon klassikl?rind?n Az?rbaycan dilin? t?rc�m? edilmi? ?eirl?r kitab? d�?d�. Tez d? �?xard?b ona g�st?rdim, Az?rbaycan dilini bilm?s? d?,  Yapon xalq?n?n klassik ?airl?rini kitabda verilmi?,  ??kill?rind?n tan?d? v? rus dilind? h?r biri haqq?nda maraql? m?lumat verdi. M?n yapon dilini bilm?s?m d? onun ?lind?ki yapon dilind? olan kitab m?ni maraqland?rd?,  ist?dim  m?nd? olan kitab? onun kita
 b?yla d?yi?m?k yolu il? ki�ik v? m?n?vi bir bazarl?q yapam. Yapon h?mkar?m buna raz? olmad? v? onda �z sad?l�hvl�y�m�z? ac???m tutdu v? yaponlar?n bu m�r?kk?b xarakterin? h?s?d apard?m. Ba??m?z s�hb?t? qar??d???ndan, b?l?d�inin, - k?m?rl?rinizi b?rkidin, be? d?qiq?dn sonra ?Domededova? hava liman?na enirik, -dem?si, m?ni yapon h?mkar?mla ?irin s�hb?td?n ay?rd?.\n Art?q u�a??m?z hava liman?nda yer? enmi?di, m?nim baqaj?m �z�md? oldu?una g�r? baqaj g�zl?m?y? vaxt itirm?y?c?kdim v? taksi dayanaca??na t?l?sdim. Cavan s�r�c�l?rd?n g�z�m su i�mirdi, ona g�r? d? ya?l? bir s�r�c�n�n taksisin? ?yl??dim, ged?c?yim �nvan? dedim. Yol boyu Moskvan?n pay?z f?slinin ?srar?ngiz g�z?lliyin? varm??d?m v? bu g�z?lliyin i�ind? bir u?aql???m gizl?nib qalm??d? sanki. Qal?n ?am a?aclar?n?n, �inar a?aclar?n?n aras?ndan uzanan yolda ?�t�y?n taksinin p?nc?r?sind?n g�c�m seyr etm?y? �at?rd? v? m?n d? bu f�rs?ti ?ld?n vermirdim. Bu an ya
 ?l? s�r�c�n�n sual? m?ni diksindirdi:\n -s?nok otkuda?\n - iz Baku.\n - oy, va?e Qeydar Aliyevi� koqo-nibud v?z?vaet sebya, oni svoim semyam pro??ayut.\n - a po�emu tak, - dey?, soru?dum,\n - oni boyitsa, potomu �to Qeydar Aliyevi� o�en stroq?y �elovek i o�en trebovatelnoy.\n M?n bu s�hb?td?n q�rur hissi ke�ir?r?k s�r�c�y?: - da, on o�en stroq?y i  velikolepniy �elovek, -dey?r?k, H. ?liyevin Az?rbaycan M?rk?zi Komit?sinin birinci katibi i?l?diyi m�dd?td? g�rd�y� i?l?rd?n dan??d?m.\n Me?? a?acla?n?n yarpaqlar? q?z?l? r?ng? �al?rd?, m?n bu �alarlar? u?aql???mda G?d?b?yin Q?xmalaxl? yayla??nda, ?vanovka k?ndinin �st yamaclar?ndak? me??l?rd?, Tovuzun Qarada?-B�y�k Q??laq istiqam?tind?ki me??l?rd? �ox ?ahidi olmu?dum. K?ndimizd?ki m?hl?mizin q?ra??ndak? k�knar a?aclar?n?n pay?z havas? is? h?mi?? m?niml?ydi. \n �atd???m �nvanda s�r�c�n�n haqq-hesab?n? ��r�d�b, sa?olla?d?m v? gec?ni pis ke�irm?dim, h?r n?sn? 
 �r?yimc? idi v? rahatland?m, ancaq bir narahatl???m var idi. Bu h?diyy?l?ri b�l�b, ayr?-ayr? torbalara qoymal?yd?m. Onda  indiki g�r�nm?y?n torba, �anta bollu?u yox idi, ancaq ka??z torbalardan istifad? ed? bil?rdim. Yax?? ki, orda i?l?y?n Valeri Yuryevi�i tan?y?rd?m v? narahat�?l??a bir ?sas olmad???n? fikirl??dim.\n S?h?r erk?n oyand?m, h?l? dan �z� yenic?  qara ilm?l?rini a?  ilm?l?rd?n saf-��r�k edib, aram-aram d?nl?yirdi. Havada ya??? iyisi yox idi, hiss olunurdu ki, pay?z g�n??i �z sar???n sa�aqlar?n? Moskvan?n k�knar me??l?rin? s?rgi ed?c?k. Bu ab-havan?n alt?nda qald???m evin eyvan?na �?xaraq, evin qar??s?ndak? park?n yarpaqlar? q?z?la �alan a?aclar?n?n ?trini, ?limd?ki siqaretl? birg? ciy?rl?rim? �?k?si oldum. Havan?n tez i??qlanmas?n? ist?yirdim, ancaq bu qara ilm?l?r, ilahi q�vv? t?r?find?n nec? toxunmu?dusa, el?c? d? qalm??d?, h?rd?n bir bir ilm? boyu a?ar?rd?. I�?ri ke�ib �ay-��r?k i?in? baxd?m, ?sas?n s?h?r yem?yimin 
 xobbisi bir tik? pendir, bir tik? d? ya? olmas?n? fikirl??ib, soyuducudan g�t�rd�m v? ruslar demi? ?zavtrak?m?? i?tahla yerin? yetirdim. Yem?kd?n sonra eyvana bir d? �?xd?m, bir siqaret yand?rd?m, art?q adamlar?n yava?-yava? evd?n �?x?b i??-g�c? ged?nl?rini seyr etm?y? ba?lad?m, art?q g?tirdiyim sovqatlar? g�t�r�b d�?m?yin vaxt? oldu?unu anlad?m. Qay?d?b, yenid?n nar v? konyak qutular?n? g�t�rd�m v? sakitc? evin dalan?na endim. Ev sahibl?ri h?l? ?irin yuxuda idi. Taksi dayanaca?? el? d? uzaq deyildi, hardasa, yuz, y�z ?lli metr olard?, ancaq ?limd?ki y�k d? y�ng�l deyildi, bu m?saf? ���n a??r oldu?unu hiss etdim. Ba?qa c�r yolu da yox idi. Burda taksi dayanaca??nda ad?t?n taksil?r durmurdu, bo? g?lib dayana bil?rdi v? ?g?r s?rni?in yoxsa, �t�b ke�m?liydi. O q?d?r d? vaxt ke�m?di g�zl?m?y?, durdu?um s?kiy? bir bo? taksi yan ald? v? tez-t?l?sik minib �nvan? bildirdim. M?saf? o q?d?r d? uzaq deyildi, bilirdim ki, 10-15 d?qiq?y? orda
 yam. \n M?n taksid?n d�??nd? art?q 20 d?qiq? olard? ki, i? ba?lam??d?. Daxili telefona ?l atd?m, Valeri Yuryevi�i soru?dum, el? telefonu g�t�r?n �z�ym�?, �z�m� t?qdim etdim, tan?d?. Ondan a?a?? d�?m?sini xahi? etdim, art?q o, yan?mda idi. V? m�?yy?n fikir m�badil?sind?n sonra m�?yy?n sayda ka??z torbalar g?tirdil?r, nar v? konyaklar? ayr?-ayr? torbalara yerl??dirib, s?kkizinci m?rt?b?y? qalxd?q v? h?min h?diyy?l?ri onun ota??na qoyduq.\n Birinci g�r�??c?yim adam t?yin oldu?um ?�b?nin k�rasiyas?n? h?yata ke�ir?n m�avinl?rd?n biri olmal?yd?. Odur ki, h?diyy? torbalar?ndan birini g�t�r�b onun ota??na yax?nla?d?m. Katib?y? kimliyimi t?qdim etdim, o da daxili telefonla ona x?b?r verdi v? i�?ri girm?yimi bildirdi. ?limd? ki, h?diyy? torbas? il? birg? i�?ri daxil oldum, baxd?m ki, torban? qoyma?a el? bir yan-y�r? yoxdu, he� otaqda mebeld?sti d? yox idi ki, bu torban? ora yerl??dirim. S?dr m�avini bunu g�zl?mirmi? kimi, m?n? baxa-bax
 a:\n -Bu n?di bel?, -dey? maraqland?.\n -bu, qafqazl?lara m?xsus h?diyy?di, -dedim.\n - apar?n q?zlara paylay?n,-dedi.\n -m?n art?q onlara vermi??m,-dey?r?k, ?limd? ki, dolu ka??z torban? bir t?h?r divar?n dibin? qoydum.\n -yax??, yax?? oturun, -dey? m?n? yer g�st?rdi.\n M?n oturdum, o, daxili d�ym?ni ??had?t barma?? il? basd?, saniy? ke�m?mi? katib? q?z i�?ri daxil oldu.\n O, y�ks?k tonla, - o torban? g�t�r�n yan?n?zda saxlay?n,- dey?r?k q?za i?ar? etdi.\n S?dr m�avini �evik v? c?ld adam idi. Hiss etm?k olurdu ki, bank sisteminin ?hali v? t??kilatlarla ba?l? hiss?sini d?rind?n bilir. H?l? xarici d�vl?tl?rd? olan texnoloji prosesl?rd?n s�hb?t getm?s? d?, daxili v? o d�vr ���n ssri ?razisi ���n yeni v? m�asir hesab edil?n primitiv texnoloji prosesl?rin h?yata ke�irilm?sind?n s�hb?t ged? bil?rdi. O, m?niml? Az?rbaycan Respublikas?n?n �mumi iqtisadiyyat?ndan s�hb?t apararaq, �mumi anlay??? ?ld? etm?y? �al??d?. M?nd? yaranan t??ss�ra
 tdan hiss etdim ki,   art?q onun raz?l???n? ?ld? etmi??m v? bu t?sdiq m?s?l?si birinci m�avinl?  g�r�??nd?n sonra �z h?llini tapacaqd?r. I�imd? top el?diyim bu hissl?rl?, onun, daxili telefonu g�t�r�b, ad? v? familyas? m?n? b?lli olan birinci m�avinl?, m?nd?n raz? halda dan??mas? �st-�st? d�?d�. Sonra o, onun yan?nda adam olub-olmamas?n? d?qiql??dirib, q?buluna g?lm?ymizi m�?yy?nl??dir?r?k, -getdik, -dey?, m?n? i?ar? etdi. \n Art?q s?drin birinci m�avinin yan?nda idik. Birinci m�avin �z �?kisin? uy?un uca boy, �ox da k�k olmayan, haz?rl?ql?, intellektual bir adam t?siri ba???lad? m?n?. El? bizi d? qar??lama??ndan hiss etdim ki, bu adamla, i? bar?sind? hans? istiqam?td? ist?s?n s�hb?t aparmaq olar. �z�m� daxil?n s?rb?st hiss etdim v? daxili narahatl???ma he� bir ?sas olmad???n? q?t etdim.\n O, s?liq?-s?hmanla v? qay??ke?likl? ba??n? qald?r?b, ?vv?lc? bizim respublikam?z?n �mumi iqtisadi v?ziyy?tiyl? maraqland?, H?s?n Qas?movi�i so
 ru?du v? onun yax?? bir s?dr olaraq, i?k�zarl???n? v?  f?aliyy?tini qiym?tl?ndirdi.\nHiss etdim ki, h?r n?sn?ni i?l? d? �l�m?k olmaz v? bu adam?n i?d?n-g�cd?n ?lav?, h?m d? �mumu b???ri keyfiyy?tl?ri d? az deyil.\n O, daxili v?ziyy?tl? yana?? avropan?n v? d�nyan?n ?n m�t?r?qqi iqtisadi yenilikl?ri �z?rind?  dayand?. Bir �ox �lk?l?rin iqtisadi r?q?ml?rind?n, texnoloji prosesl?rind?n misallar �?kdi. Ancaq onda bunlar nisbi inki?a g�st?ricil?riydi, bu g�st?ricil?rin alt?nda xaricd? hans? texnoloji prosesl?rin durdu?u n? onun ���n, n? d? m?nim ���n anla??lan deyildi. T?kc? biz onu bilirdi ki, ?g?r Amerikada texnoloji prosesin bu v? ya dig?r sah?si 80-90 faiz t??kil edirdi s? bu ssrid? 10-20 faz �evr?sind? ola bil?rdi v? bu m�qayis?l?r bu istiqam?td? �z �l��s�n� v? geriliyini m�?yy?n ed? bil?rdi.\n B�t�n bunlara baxmayaraq, m?n ona diqq?tl? qulaq as?rd?m, ��nki bunlar m?nim ���n maraql? v? �n?mli idi. Bu �n?mli fak
 torlar?n  arxas?nda m?nim yeni i? f?aliyy?tim dururdu  v? bu i?l?ri yeni �sulda h?yata ke�irm?k ���n  �?kil?n v? g�st?ril?n misallar?n t?cr�b?d? istifad? imkanlar? m�tl?q v? laz?ml? idi, fikirl??dim.  H?r halda birinci m�avinin m?niml?, o d�vr�n diliyl? des?k, apard??? s�hb?tl?r, ataca??m add?mlar?n b�n�vr?si ���n  inaml? bir t?kan hesab edil? bil?rdi.\n Onun raz?l???n? v? xeyir-duas?n? ald?dan sonra, m?nd? yaranm?? xo? ?hval-ruhiyy? il? yan?ndan �?xd?m, n?z?rd? tutulmu? idar? v? ?�b? r?isl?riyl? g�r�?d�m, s�hb?t etdim, laz?m? tap??r?qlar? ald?m v? laz?m? s?n?dl?r? d?rk?narlar? qoydurdum. Art?q yeni v?zif?y? t?yinat?m bar?d? ?mrim haz?r idi, birc? qal?rd?  geriy? d�n�b, h?v?sl? i?? ba?lama??m.\n I?in sonuna lap az qalm??d?, fikirl??dim ki, Valeri Yuryevi�l? g�r�?m?liy?m, o da xo?b?xtlikd?n ba?qa bir i?in dal?nca getm?mi?di, yenid?n s?kkizinci m?rt?b?y? qalxd?m. Valeri Yuryevi� balacaboy, i? t?cr�b?sin? malik v? qay??ke? b
 ir adam idi, h?m d? m?nim t?yin oldu?um ?�b?nin, ssri �zr? sah? r?isi idi. H?mya??d olard?q. Ota??nda t?k idi. Ota??na daxil olub sevincimi onunla b�l�?d�rd�m, bir xeyli s�hb?t etdik, i?in maraql? olaca??n? bildirdi. M?nim beynimd? art?q i?d?n ?s?r ?lam?t qalmam??d?, birc? o qalm??d? ki, i?in sonu n? vaxt g?lib �atacaq v? bu sevinci, Valeri il? Moskvan?n M.Qorki k��?sind? yerl???n Bak? restoran?nda bir daha ??r?blana-??rablana b�l�?d�r?�?yik. Bel? anlarda, t?rs kimi  be?-on d?qiq?nin g?lib getm?si d? adama ���n dar?xd?r?c? olurmu?. Yax?? ki, s�hb?t maraql? al?nd? v? vaxt g?lib ke�di, s�hb?tin arxas?n? Bak? restoran?nda davam etdirm?yi q?rara ald?m. Onun m?nim niyy?timd?n x?b?ri yox idi. O, ona g?tirdiyim h?diyy?ni g�t�rm?k ist?y?nd?, bildirdim ki, onlar? mebel d?stinin g�z�nd? yerl??dirsin v? m?niml? getm?sini xahi? etdim. O, ?vv?l t?r?dd�t ets? d? sonra m?niml? raz?la?m?? formada otaqdan birg? �?xd?q. Bak? restoran? el? d? uzaq de
 yildi, i? yerind?n  g?z?-g?z? gets?k uzaqba?? on d?q?q?y? orda ola bil?rdik. Ancaq buna baxmayaraq, taksi yaxalad?m v? tez d? mindik, t?xmin?n iki d?qiq?d?n sonra restoran?n yan?nda idik. Restoran?n qap?s?nda duran ??vetsar?  yerl?rin tutldu?unu v? g�zl?m?ymizi bildirdi, y?ni yax?n d?qiq?l?rd? stolun birinin bo?alaca??n? bildik. Restoran?n gir?c?yind? var-g?l etdik, zaman da zaman idi, laz?m olan anlarda ir?lil?m?k bilmir ki, bilmir. N?hy?t, ??vetsar??n g�st?ri?in? ?sas?n bo? olan stola yax?nla?d?q, ?trafda qonaqlar �ox idi, ?sas?n d? az?rbaycanl?lar idi. G�z g?zdirs?m d? tan?? adam g�z�m? d?ym?di. \n Sifari?l?rimiz, m�vafiq raz?la?madan sonra Bak? m?tb?xin? aid kabablar v? ba?qa t?amlar oldu. Sa?l?qlar qar??l?ql? olsa da,  m?nim yeni v?zif? t?yinat?m ���n Valeri ?hat?li v? geni? nitq  etdi. Onun nitqind? ki, s?mimiyy?tin qanadlar? �iynimd?n as?la qalm??d?, ��nki sabahlar? i?in ir?lil?yi?i il? ba?l? hesabatlar? onun ?�b?sin? t?qdim ed?ciyimi bilirdim. Odu
 r ki, k�n�ls�z d? olsa, �z d?rin minn?tdarl???m? bildirdim. Art?q durma??n vaxt?yd?, vaxt gec?ni ?ymi?di, restoranda t?k?m-seyr?k adam qalm??d?, onlar da hallanan adamlara ox?ay?rd?. Stola xidm?t ed?n qulluq�u, restoran?n ba?lan?lmas?na be? d?qiq? qald???n? bildirdi. Stolun haqq-heab?n? yekunla?d?r?b restorandan �?xd?q, t?k?m-seyr?k taksil?r g�z? d?yirdi, el? bil t?l?sidiymizi bilirdi, biri saxlad?. �yr?ndim ki, Valeri d? m?n getdiyim istiqam?td? ya?ay?r. M?n Dimitrovski ?ossesinin ax?rna �atmam?? d�?m?li idim, o is? ax?r?nda.  N?zak?t xatirin? onu evin? kimi �t�r?si oldum v? h?min taksiyl? d?, yenid?n �z qald???m �nvana g?ldim.\n Qald???m �nvan yad deyildi, n? d? kiray? deyildi, uzun m�dd?t Moskvan?n sakini olmu? b�y�k qarda??m?m eviydi. Qarda??m da m?ni g�zl?diyind?n h?l? yatmam??d?. Yeni v?zif?mi   bir d? ??rablan?b t?krarlad?q. T?brik v? sa?l?qlardan sanra yetmi?inci ill?rd? burda bir pay?z vaxt? istirah?t ed?rk?n yazd???m ?eir yad?ma d�?d
 �. Ba?lad?m onu q?r?q-q?r?q yad?ma salma?a v? �z i�imd? p?�?ldama?a:\n Moskva...\n Bo? otaq...\n T?k?m, t?nhayam,\n Rusiyan?n bom-boz pay?z? kimi,\n Radio dal?as?nda ?Bak?? g?zm?kd?n\n Dar?xd?, titr?di q?lbimin simi.\n\n Moskva...\n Bo? otaq...\n Dinir ?T�rkiy??,\n Nazimin ?eiri, ??rqisi bu da,\n N? yaman uyudum bura harad?r,\n A? sa�l? Qorqudu g?tirdi yada.\n\n Moskva...\n Bo? otaq...\n �r?yimd? d?,\n Tar?ma �?kilmi? t?qvim ill?ri.\n Eh, bax, bu ?eir d? s?n? �atmad?,\n Soldu misralar?, soldu g�ll?ri.\n\n Iqlim d?yi?s? d? t?bi?timd?,\n Dilimi, s�z�m� yen? bilmir?m.\n Dem? ?eirliyim bura kimiymi?,\n Qagqazdan aral? din? bilmir?m.\n \n 1976-c? ild? Moskvada, bax, el? bu evd? yazd???m bu ?eirin yad?mda s?ralanm?? bu b?ndl?rini bir t?h?r v? bir daha,  dilim topuq vura-vura qarda??m ���n d? oxudum.  Gec?ni sevinc?k ke�irdik. Dan �z�n? az qalm??d?. Fikirl??dim ki, g�z�m�n hovurnu al?m, u�aq aya??na gecikm?yim, h?m d? bilet  m?s?l?si d? var idi
 . Oydu ki, yalandan da olsa divana uzand?m, ??rab?n meyxo?lu?undan yuxuya gets?m d?, stol�st� saat?n z?ngin? oyand?m. Saat  on r?q?minin d?qiq?l?rini yenic? add?mlama?a ba?lam??d?, h?l? ki, on be?inci add?m?ndayd?. �z�m� vanna ota??na sald?m, �z�m?, g�z�m? isti-soyuq qar???q su vurdum, d?smallan?b, vanna ota??ndan �?xd?m.  Qarda??m? oyatma?a ?lim g?lm?s? d?, m?cburuydum oyatma?a, ��nki qap?n?n a�?lmas? v? ba?lan?lmas? var?yd?. O, bunu hiss etmi? kimi, getm?yimi soru?du, bir st?kan p�rr r?ngi �ay i�ib, bozarm?? hvada taksi dayanaca??na t?l?sdim v? bir taksiy? oturub hava liman?na y�n ald?m.\n Hava liman?n?n bilet kassas?nda n�vb? �ox idi, dayand?m, n?h?y?t ki, n�vb?m �atd?. Moskvadan Bak?ya u�an u�a??n n�vb?ti reysin? bilet ald?m. H?l? vaxta var idi. Yenid?n ax?amk? mar?urutu bir d? t?krar ed?si oldum. M.Qorki k��?siyl? �z� yuxar? yolland?m. Qar??mda yen? ?Bak??restoran?yd?. G�nd�z oldu?undan restoranda adam az idi. Bo? stol
 lar?n birin? yax?nla??b ( onsuz da stollar  bo? idi),\nOturdum v? menyunu g�zd?n ke�irm?y? ba?lad?m.  Az?rbaycan m?tb?xin? aid h?r n?sn? il? z?ngin idi  restoran. \n Ba??m? qald?rd?m ki, qar??mda bir adam durub, ancaq tan?mad?m. Ad?m? �?k?r?k, -salam, tan?mad?n m?ni?, -dey?r?k, m?ni s�zd�.\n -tan?? g?lirsiniz, ancaq xat?rlaya bilmir?m, ad?n?z da yad?mdan �?x?b, -dedim.\n -bir yax?? fikirl?? Asif ?f?ndiyevl? s?nd?n ba?qa kiml?r yax?nl?q edirdi.\n - h?, h?, -dedim,-yad?ma d�?d�, siz K�rd?mird?n idiniz, ad?n?z da Z?man?t, famiyan?z da B?kirov.  Siz Moskvaya aspiranturaya g?ldiniz, s?hv etmir?ms?, m�vzunuz da ?Heygel f?ls?f?sind? ??rq m�vzusu?  idi. �z� d? s?hv etmir?ms?, �. Hac?b?yov ad?na Bak? D�vl?t Konservatoriyas?n?n ?ahnaz adl? bir m�?llim?si var idi siz onunla birg? g?ldiniz. O vaxtlar deyirdiniz ki, sizi �� f?lak?t g�zl?yir, bax, indi g�r�r?m ki, he� bir f?lak?tiniz filan yoxdu. Y?qin ki, m�dafi?nizi d? etmisiniz.\n O, d?rind?n k�k
 s �t�rd�, g�l�ms�nd� v? soru?du, - de g�r�m yem?k sifari?i vermis?n, ya yox, s�hb?timizi rahat ed?k, -dedi.\n Stola qulluq ed?n a? xalatl? q?z, sanki bizi g�rm�rd�, b?lk? d? s�hb?t etdiyimizi g�r�b rahat buraxm??d? ki, yan?mdak? adam da otursun v? sonra biz? yax?nla?s?n. \n N?hay?t ki, stola qulluq ed?n a? xalatl? q?z biz? yax?nla?d?. Art?q m?n? ayd?n idi ki, m?nim bu k�hn? dostum da restorana indi daxil olub v? �mumi bir sifari? ver?si olduq. Ancaq u�aq aya??nda olsam da k�hn? dostun x?tirn? d?y? bilm?dim, G?nc? konyak?n? t?rifl?di v? s�hb?timizi d? G?nc? konyak? alt?nda ba?lad?q.\n O, �z�n� m?n? tutub, - h?, o dediyin f?lak?tin ikisi oldu da, qurtard? da. ���nc� f?lak?ti g�zl?yir?m, n?d?n ibar?t olaca??n? a�?qlaya bilm?y?c?y?m, g?r?k �z�rl� hesab ed?siniz, ancaq vaxt? g?l?nd? dey?r?m. O dediyin m�vzunu m�dafi? etmi??m, hal-haz?rda doktorluk dissertasiyas? �z?rind? �al???ram, ancaq d?y?s?n �m�r buna imkan
  verm?y?c?k. ?ahnaz da ?milli m�siqimizin f?ls?f?sind?n? m�dafi? etdi v? hal-haz?rda   Bak? D�vl?t Konservatoriyas?nda m�?llim i?l?yir. H?l? ki, Bak?ya gec-gec gedir?m. A. ?f?ndiyevi g�rm�rs?n ki, onun da taleyi d�z g?tirm?di.  Az?rbaycanda A. ?f?ndiyev �ox m�tail?li bir filosofdu, m?n f?ls?f?nin ?n d?rin qatlar?n? ondan �yr?nmi??m. El? Heygel f?ls?f?sind? ??rq m�vzusunu da o, m?nim a?l?ma salm??d?. Ancaq A. ?f?ndiyevin bir �at??mayan c?h?ti oldu ki, f?ls?f?ni sisteml??dir? bilm?di, bu da onun t?k?bb�r� il? ba?l? bir m?s?l? idi. M?n h?mi?? onunla s�hb?tl?rimd? bu m?s?l?l?ri ona xat?rlad?rd?m v? xahi? edirdim ki, b�t�n bu fikirl?ri bir m?craya g?tirsin, f?ls?f?ni yeni istiqam?td? �mumil??dirsin v? �mumi bir n?z?riyy? hal?na sals?n. O, m?niml? raz?la?m??d?, ancaq �z�n� toparlama?a vaxt? yox idi, ��nki h?mi?? g?ncl?r aras?nda ?oca??? m�q?dd?sl??dir?r?k, bu oca??n ?�l?sini h?r ?eyd?n �st�n tuturdu. O, f?ls?f?ni V?t?n sevgisind?, oc
 aq sevgisind?, b�y�k t�rkl�y�n yenilm?zliyind?, ad?t?t ?n?n?sind? g�r�rd� v? vaxt?n? bu g�r�?l?r? h?sr etdiyind?n b?zi m?s?l?l?r bitm?mi? qald?, buna heyifsl?nm?kd?n ba?qa �ar?miz yoxdu. \n M?n maraql? m�vzuya m�daxil? ed?r?k, - Asif ?f?ndiyevd? G�y Tanr? inanc? var idi, o, yerin d?, g�y�n d?, canl? v? cans?z?n da ilahi bir  q�vv? t?rfind?n yarad?ld???n?n izah etdiyinin sizinl? birg? ?ahidiyik. Bir t�rk olaraq saz?m?z? da b�t�n t�rk d�nyas?n?n bayra?? elan etmi?di, qalan al?tl?ri is? g?lm? hesab edirdi.Az?rbaycan D�vl?t Universitetind? ke�irdiyi b?dii-f?ls?fi gec?l?r?, saz-s�z ustas?, ?mran H?s?nov, ?kb?r C?f?rov, ?dal?t N?sibov ?Misrisi?yl?, ?C?ngi?siyl?, ?Dilq?mi?siyl?, ?Ruhani?siyl?, ?Q?hraman??s?yla, Az?rbaycan?n g�rk?mli xan?nd?l?rind?n Yaqub M?mm?dov, Can?li ?kb?rov ?M?nsuriyy??siyl?, ??anaz??yla, Kamil C?lilov q?boyundak? inc? ritml?riyl? qat?lard?. �mumiyy?tl? o, b�t�n f?ls?f?nin d�y�n n�qt?sini T�rk D�nyas?
  adl? b�y�k bir co?rafiyan?n m�siqisind?, bayat?s?nda, a??s?nda, ac?s?nda g�r?rdi. Ancaq m?n d? sizinl? raz?yam ki, o, f?ls?f?ni bu co?rafi ?razi �zr? sisteml??dir? bilm?di, bu da ssri kimi bir imperiyan?n ideologiyas? il? m�bariz?y? s?rf olunan zaman itgisiyl? ba?l?yd?. Tanr? ona m�st?qilliyimizd?n sonra laz?m? �m�r v? ya?am ���n m�vafiq t?minat vers?ydi, y?qin ki, o, f?ls?f?nin m�?yy?n sah?l?rini �mumil??dir? bil?c?kdi.\n S�hb?t? ba??m?z qar??d???ndan yem?kl?rin stola d�z�ld�y�nd?n x?b?rimiz yox idi. G�z�m  ya??la �alan ?�?? butulkada g?tirilmi? G?nc? konyak?na sata?anda Z?man?tin �z�n? baxd?m. O, m?ni hiss edirmi?  kimi, ?�?? butulkaya ?l atd?, ilk bad?l?ri t?sad�fi g�r�?m�z�n daimi olmas? ??r?fin? nu? etdik. N�vb?ti  bad?l?rin d? s?b?bkarlar? tap?ld?  v? nu? edib qalxd?q. Hesab? is? Z?man?t ver?si oldu. O, bunu onunla izah etdi ki, o m?nd?n b�y�kd� v? m?n Moskvada qona?am, o da ba?a d�?�l?n bir n?sn? idi
 . \n O, m?nim u�u? vaxt?ma az qald???n? hiss edib, m?ni �t�rm?yi, yola salma??  q?rara ald?, m?n buna raz? olmad?m. O, qara kiprikl?rinin v? qara b??lar?n?n alt?ndan b?rq vuran t?b?ss�ml� g�l�?l?riyl?, m?ni, el? restoran?n h?y?tind?n yola sald? v? x�dahifizl??ib ayr?ld?q.\n Bir ba?a hava liman?na g?ldim, u�u?un vaxt?na h?l? bir saat qalm??d?."}},
-{"photo":{"from":{"name":"Thami de Oliveira","id":"100003470164026"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yz/r/StEh3RhPvjk.gif","id":"100003470164026_390971624361840","created_time":"2013-08-28T00:26:45+0000","type":"photo","updated_time":"2013-08-28T00:30:15+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=390971007695235&set=pcb.390971624361840&type=1&relevant_count=3","object_id":"390971007695235","message":"MENINAS QUE TIVEREM INTERESSE NESSE VESTIDO TEM NA COR BRANCO, AMARELO, AZUL BIC, VERDE E PRETO, COMENTA COM A COR QUE FOR QUERER PRA EU PODER FAZER A ENCOMENDA, S� VOU TRAZER DE QUEM COMENTAR !!!","picture":"http://photos-h.ak.fbcdn.net/hphotos-ak-prn1/1236641_390971007695235_471308162_t.jpg"}},
-{"photo":{"name":"Timeline Photos","story":"Luis Carlos shared Minha V� Ta Maluca's photo.","from":{"name":"Luis Carlos","id":"100003127462424"},"application":{"name":"Photos","id":"2305272732"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"0":[{"name":"Luis Carlos","id":"100003127462424","type":"user","length":11,"offset":0}],"19":[{"name":"Minha V� Ta Maluca","id":"282072935240952","type":"page","length":18,"offset":19}]},"id":"100003127462424_439260802854831","created_time":"2013-08-28T00:26:05+0000","type":"photo","caption":"Recomendo : Sem Sono\r\nRecomendo : Eu Amo Viajar","updated_time":"2013-08-28T00:26:05+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=468887093226201&set=a.445796168868627.1073741872.282072935240952&type=1","object_id":"468887093226201","message":". . . . ja, fui bic�o em varias, e ainda chegava comprimentando todo mundo . . . . ","properties":[{"href":"http://www.facebook.com/votamaluc
 a?ref=stream","name":"By","text":"Minha V� Ta Maluca"}],"picture":"http://photos-a.ak.fbcdn.net/hphotos-ak-ash3/1256526_468887093226201_1763899420_s.jpg"}},
-{"photo":{"is_hidden":true,"name":"Timeline Photos","story":"Giovanna Lopes shared 4:20's photo.","from":{"name":"Giovanna Lopes","id":"100003553147558"},"application":{"name":"Links","id":"2309869772"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"22"

<TRUNCATED>

[13/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/api/FacebookPostActivitySerializer.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/api/FacebookPostActivitySerializer.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/api/FacebookPostActivitySerializer.java
new file mode 100644
index 0000000..f1687ea
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/api/FacebookPostActivitySerializer.java
@@ -0,0 +1,286 @@
+/*
+ * 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.facebook.api;
+
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.streams.data.ActivitySerializer;
+import org.apache.streams.exceptions.ActivitySerializerException;
+import org.apache.streams.facebook.Post;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.pojo.json.*;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.streams.data.util.ActivityUtil.*;
+
+/**
+ * Serializes activity posts
+ *   sblackmon: This class needs a rewrite
+ */
+public class FacebookPostActivitySerializer implements ActivitySerializer<org.apache.streams.facebook.Post> {
+
+    public static final DateTimeFormatter FACEBOOK_FORMAT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ssZ");
+    public static final DateTimeFormatter ACTIVITY_FORMAT = ISODateTimeFormat.basicDateTime();
+
+    public static final String PROVIDER_NAME = "facebook";
+
+    public static ObjectMapper mapper;
+    static {
+        mapper = StreamsJacksonMapper.getInstance();
+    }
+
+    @Override
+    public String serializationFormat() {
+        return "facebook_post_json_v1";
+    }
+
+    @Override
+    public Post serialize(Activity deserialized) throws ActivitySerializerException {
+        throw new NotImplementedException("Not currently supported by this deserializer");
+    }
+
+    @Override
+    public Activity deserialize(Post post) throws ActivitySerializerException {
+        Activity activity = new Activity();
+        activity.setPublished(post.getCreatedTime());
+        activity.setUpdated(post.getUpdatedTime());
+        addActor(activity, mapper.convertValue(post.getFrom(), ObjectNode.class));
+        setProvider(activity);
+        setObjectType(post.getType(), activity);
+        parseObject(activity, mapper.convertValue(post, ObjectNode.class));
+        fixObjectId(activity);
+        fixContentFromSummary(activity);
+        return activity;
+    }
+
+    @Override
+    public List<Activity> deserializeAll(List<Post> serializedList) {
+        throw new NotImplementedException("Not currently supported by this deserializer");
+    }
+
+    private void fixContentFromSummary(Activity activity) {
+        //we MUST have a content field set, so choose the best option
+        if(activity.getContent() == null) {
+            activity.setContent(activity.getAdditionalProperties().containsKey("summary") ?
+                    (String) activity.getAdditionalProperties().get("summary") :
+                    activity.getObject().getSummary());
+        }
+    }
+
+    private void fixObjectId(Activity activity) {
+        //An artifact of schema generation, the default value is {link}
+        if(activity.getObject().getId().equals("{link}")) {
+            activity.getObject().setId(null);
+        }
+    }
+
+    private void setObjectType(String type, Activity activity) {
+        ActivityObject object = new ActivityObject();
+        activity.setObject(object);
+        object.setObjectType(type);
+    }
+
+    private void setProvider(Activity activity) {
+        Provider provider = new Provider();
+        provider.setId(getProviderId(PROVIDER_NAME));
+        provider.setDisplayName(PROVIDER_NAME);
+        activity.setProvider(provider);
+    }
+
+    private String getObjectType(JsonNode node) {
+        Iterator<Map.Entry<String, JsonNode>> fields = node.fields();
+        ensureMoreFields(fields);
+        Map.Entry<String, JsonNode> field = fields.next();
+        //ensureNoMoreFields(fields);
+        return node.asText();
+    }
+
+    private void parseObject(Activity activity, JsonNode jsonNode) throws ActivitySerializerException {
+        for(Iterator<Map.Entry<String, JsonNode>> fields = jsonNode.fields(); fields.hasNext();) {
+            Map.Entry<String, JsonNode> field = fields.next();
+            String key = field.getKey();
+            JsonNode value = field.getValue();
+            mapField(activity, key, value);
+        }
+    }
+
+    private void mapField(Activity activity, String name, JsonNode value) throws ActivitySerializerException {
+        if("application".equals(name)) {
+            addGenerator(activity, value);
+        } else if ("caption".equals(name)) {
+            addSummary(activity, value);
+        } else if ("comments".equals(name)) {
+            addAttachments(activity, value);
+        } else if ("description".equals(name)) {
+            addObjectSummary(activity, value);
+        } else if ("from".equals(name)) {
+            addActor(activity, value);
+        } else if ("icon".equals(name)) {
+            addIcon(activity, value);
+        } else if ("id".equals(name)) {
+            addId(activity, value);
+        } else if ("is_hidden".equals(name)) {
+            addObjectHiddenExtension(activity, value);
+        } else if ("like_count".equals(name)) {
+            addLikeExtension(activity, value);
+        } else if ("link".equals(name)) {
+            addObjectLink(activity, value);
+        } else if ("message".equals(name)) {
+            activity.setContent(value.asText());
+        } else if ("name".equals(name)) {
+            addObjectName(activity, value);
+        } else if ("object_id".equals(name)) {
+            addObjectId(activity, value);
+        } else if ("picture".equals(name)) {
+            addObjectImage(activity, value);
+        } else if ("place".equals(name)) {
+            addLocationExtension(activity, value);
+        } else if ("shares".equals(name)) {
+            addRebroadcastExtension(activity, value);
+        } else if ("source".equals(name)) {
+            addObjectLink(activity, value);
+        } else if ("story".equals(name)) {
+            addTitle(activity, value);
+        }
+    }
+
+    private void addSummary(Activity activity, JsonNode value) {
+        activity.setAdditionalProperty("summary", value.asText());
+    }
+
+    private void addTitle(Activity activity, JsonNode value) {
+        activity.setTitle(value.asText());
+    }
+
+    private void addLikeExtension(Activity activity, JsonNode value) {
+        Map<String, Object> extensions = ensureExtensions(activity);
+        extensions.put(LIKES_EXTENSION, value.asInt());
+    }
+
+    private void addLocationExtension(Activity activity, JsonNode value) {
+        Map<String, Object> extensions = ensureExtensions(activity);
+        if(value.has("location")) {
+            Map<String, Object> location = new HashMap<String, Object>();
+            JsonNode fbLocation = value.get("location");
+            if(fbLocation.has("country")) {
+                location.put(LOCATION_EXTENSION_COUNTRY, fbLocation.get("country"));
+            }
+            if(fbLocation.has("latitude") && fbLocation.has("longitude")) {
+                location.put(LOCATION_EXTENSION_COORDINATES, String.format("%s,%s", fbLocation.get("longitude"), fbLocation.get("latitude")));
+            }
+            extensions.put(LOCATION_EXTENSION, location);
+        }
+    }
+
+    private void addObjectImage(Activity activity, JsonNode value) {
+        Image image = new Image();
+        image.setUrl(value.asText());
+        activity.getObject().setImage(image);
+    }
+
+    private void addObjectId(Activity activity, JsonNode value) {
+        activity.getObject().setId(getObjectId("facebook", activity.getObject().getObjectType(), value.asText()));
+    }
+
+    private void addObjectName(Activity activity, JsonNode value) {
+        activity.getObject().setDisplayName(value.asText());
+    }
+
+    private void addId(Activity activity, JsonNode value) {
+        activity.setId(getActivityId(PROVIDER_NAME, value.asText()));
+    }
+
+    private void addObjectLink(Activity activity, JsonNode value) {
+        activity.getObject().setUrl(value.asText());
+    }
+
+    private void addRebroadcastExtension(Activity activity, JsonNode value) {
+        Map<String, Object> extensions = ensureExtensions(activity);
+        if(value.has("count")) {
+            extensions.put(REBROADCAST_EXTENSION, value.get("count").asInt());
+        }
+    }
+
+    private void addObjectHiddenExtension(Activity activity, JsonNode value) {
+        Map<String, Object> extensions = ensureExtensions(activity);
+        extensions.put("hidden", value.asBoolean());
+    }
+
+    private void addIcon(Activity activity, JsonNode value) {
+        Icon icon = new Icon();
+        //Apparently the Icon didn't map from the schema very well
+        icon.setAdditionalProperty("url", value.asText());
+        activity.setIcon(icon);
+    }
+
+    private void addActor(Activity activity, JsonNode value) {
+        Actor actor = new Actor();
+        if(value.has("name")) {
+            actor.setDisplayName(value.get("name").asText());
+        }
+        if(value.has("id")) {
+            actor.setId(getPersonId(PROVIDER_NAME, value.get("id").asText()));
+        }
+        activity.setActor(actor);
+    }
+
+    private void addObjectSummary(Activity activity, JsonNode value) {
+        activity.getObject().setSummary(value.asText());
+    }
+
+    private void addGenerator(Activity activity, JsonNode value) {
+        Generator generator = new Generator();
+        if(value.has("id")) {
+            generator.setId(getObjectId(PROVIDER_NAME, "generator", value.get("id").asText()));
+        }
+        if(value.has("name")) {
+            generator.setDisplayName(value.get("name").asText());
+        }
+        if(value.has("namespace")) {
+            generator.setSummary(value.get("namespace").asText());
+        }
+        activity.setGenerator(generator);
+    }
+
+    private void addAttachments(Activity activity, JsonNode value) {
+        //No direct mapping at this time
+    }
+
+    private static void ensureMoreFields(Iterator<Map.Entry<String, JsonNode>> fields) {
+        if(!fields.hasNext()) {
+            throw new IllegalStateException("Facebook activity must have one and only one root element");
+        }
+    }
+    private static void ensureNoMoreFields(Iterator<Map.Entry<String, JsonNode>> fields) {
+        if(fields.hasNext()) {
+            throw new IllegalStateException("Facebook activity must have one and only one root element");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
new file mode 100644
index 0000000..a44f982
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
@@ -0,0 +1,29 @@
+/*
+ * 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.facebook.feed;
+
+/**
+ * Created with IntelliJ IDEA.
+ * User: sblackmon
+ * Date: 10/2/13
+ * Time: 6:32 PM
+ * To change this template use File | Settings | File Templates.
+ */
+public class FacebookPublicFeedXmlActivitySerializer {
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/processor/FacebookTypeConverter.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/processor/FacebookTypeConverter.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/processor/FacebookTypeConverter.java
new file mode 100644
index 0000000..2c6fd8e
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/processor/FacebookTypeConverter.java
@@ -0,0 +1,194 @@
+/*
+ * 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.facebook.processor;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.Lists;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProcessor;
+import org.apache.streams.exceptions.ActivitySerializerException;
+import org.apache.streams.facebook.Post;
+import org.apache.streams.facebook.api.FacebookPostActivitySerializer;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.pojo.json.Activity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * Created by sblackmon on 12/10/13.
+ */
+public class FacebookTypeConverter implements StreamsProcessor {
+
+    public final static String STREAMS_ID = "TwitterTypeConverter";
+
+    private final static Logger LOGGER = LoggerFactory.getLogger(FacebookTypeConverter.class);
+
+    private ObjectMapper mapper;
+
+    private Queue<StreamsDatum> inQueue;
+    private Queue<StreamsDatum> outQueue;
+
+    private Class inClass;
+    private Class outClass;
+
+    private FacebookPostActivitySerializer facebookPostActivitySerializer;
+
+    private int count = 0;
+
+    public final static String TERMINATE = new String("TERMINATE");
+
+    public FacebookTypeConverter(Class inClass, Class outClass) {
+        this.inClass = inClass;
+        this.outClass = outClass;
+    }
+
+    public Queue<StreamsDatum> getProcessorOutputQueue() {
+        return outQueue;
+    }
+
+    public void setProcessorInputQueue(Queue<StreamsDatum> inputQueue) {
+        inQueue = inputQueue;
+    }
+
+    public Object convert(ObjectNode event, Class inClass, Class outClass) throws ActivitySerializerException, JsonProcessingException {
+
+        Object result = null;
+
+        if( outClass.equals( Activity.class )) {
+            LOGGER.debug("ACTIVITY");
+            result = facebookPostActivitySerializer.deserialize(mapper.convertValue(event, Post.class));
+        } else if( outClass.equals( Post.class )) {
+            LOGGER.debug("POST");
+            result = mapper.convertValue(event, Post.class);
+        } else if( outClass.equals( ObjectNode.class )) {
+            LOGGER.debug("OBJECTNODE");
+            result = mapper.convertValue(event, ObjectNode.class);
+        }
+
+        // no supported conversion were applied
+        if( result != null ) {
+            count ++;
+            return result;
+        }
+
+        LOGGER.debug("CONVERT FAILED");
+
+        return null;
+
+    }
+
+    public boolean validate(Object document, Class klass) {
+
+        // TODO
+        return true;
+    }
+
+    public boolean isValidJSON(final String json) {
+        boolean valid = false;
+        try {
+            final JsonParser parser = new ObjectMapper().getJsonFactory()
+                    .createJsonParser(json);
+            while (parser.nextToken() != null) {
+            }
+            valid = true;
+        } catch (JsonParseException jpe) {
+            LOGGER.warn("validate: {}", jpe);
+        } catch (IOException ioe) {
+            LOGGER.warn("validate: {}", ioe);
+        }
+
+        return valid;
+    }
+
+    @Override
+    public List<StreamsDatum> process(StreamsDatum entry) {
+
+        StreamsDatum result = null;
+
+        try {
+
+            Object item = entry.getDocument();
+            ObjectNode node;
+
+            LOGGER.debug("{} processing {}", STREAMS_ID, item.getClass());
+
+            if( item instanceof String ) {
+
+                // if the target is string, just pass-through
+                if( String.class.equals(outClass)) {
+                    result = entry;
+                }
+                else {
+                    // first check for valid json
+                    node = (ObjectNode)mapper.readTree((String)item);
+
+                    // since data is coming from outside provider, we don't know what type the events are
+                    // for now we'll assume post
+
+                    Object out = convert(node, Post.class, outClass);
+
+                    if( out != null && validate(out, outClass))
+                        result = new StreamsDatum(out);
+                }
+
+            } else if( item instanceof ObjectNode || item instanceof Post) {
+
+                // first check for valid json
+                node = (ObjectNode)mapper.valueToTree(item);
+
+                // since data is coming from outside provider, we don't know what type the events are
+                // for now we'll assume post
+
+                Object out = convert(node, Post.class, outClass);
+
+                if( out != null && validate(out, outClass))
+                    result = new StreamsDatum(out);
+
+            }
+
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        if( result != null )
+            return Lists.newArrayList(result);
+        else
+            return Lists.newArrayList();
+    }
+
+    @Override
+    public void prepare(Object o) {
+        mapper = new StreamsJacksonMapper();
+        facebookPostActivitySerializer = new FacebookPostActivitySerializer();
+    }
+
+    @Override
+    public void cleanUp() {
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendFeedProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendFeedProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendFeedProvider.java
new file mode 100644
index 0000000..381e6f3
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendFeedProvider.java
@@ -0,0 +1,282 @@
+/*
+ * 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.facebook.provider;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookFriendFeedProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookFriendFeedProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendFeedProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
+
+    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
+
+    protected Iterator<String[]> idsBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookFriendFeedProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookFriendFeedProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        shutdownAndAwaitTermination(executor);
+        running.set(true);
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        StreamsResultSet current;
+
+        synchronized (FacebookUserstreamProvider.class) {
+            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
+            current.setCounter(new DatumStatusCounter());
+            current.getCounter().add(countersCurrent);
+            countersTotal.add(countersCurrent);
+            countersCurrent = new DatumStatusCounter();
+            providerQueue.clear();
+        }
+
+        return current;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+        Facebook client = getFacebookClient();
+
+        try {
+            ResponseList<Friend> friendResponseList = client.friends().getFriends();
+            Paging<Friend> friendPaging;
+            do {
+
+                for( Friend friend : friendResponseList ) {
+
+                    executor.submit(new FacebookFriendFeedTask(this, friend.getId()));
+                }
+                friendPaging = friendResponseList.getPaging();
+                friendResponseList = client.fetchNext(friendPaging);
+            } while( friendPaging != null &&
+                    friendResponseList != null );
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(configuration.getOauth().getAppId())
+            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFriendFeedTask implements Runnable {
+
+        FacebookFriendFeedProvider provider;
+        Facebook client;
+        String id;
+
+        public FacebookFriendFeedTask(FacebookFriendFeedProvider provider, String id) {
+            this.provider = provider;
+            this.id = id;
+        }
+
+        @Override
+        public void run() {
+            client = provider.getFacebookClient();
+                try {
+                    ResponseList<Post> postResponseList = client.getFeed(id);
+                    Paging<Post> postPaging;
+                    do {
+
+                        for (Post item : postResponseList) {
+                            String json = DataObjectFactory.getRawJSON(item);
+                            org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                            try {
+                                lock.readLock().lock();
+                                ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                                countersCurrent.incrementAttempt();
+                            } finally {
+                                lock.readLock().unlock();
+                            }
+                        }
+                        postPaging = postResponseList.getPaging();
+                        postResponseList = client.fetchNext(postPaging);
+                    } while( postPaging != null &&
+                            postResponseList != null );
+
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendUpdatesProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendUpdatesProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendUpdatesProvider.java
new file mode 100644
index 0000000..2a5ec65
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookFriendUpdatesProvider.java
@@ -0,0 +1,286 @@
+/*
+ * 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.facebook.provider;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookFriendUpdatesProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookFriendPostsProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendUpdatesProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
+
+    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
+
+    protected Iterator<String[]> idsBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookFriendUpdatesProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookFriendUpdatesProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        running.set(true);
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        Preconditions.checkArgument(idsBatches.hasNext());
+
+        LOGGER.info("readCurrent");
+
+        // return stuff
+
+        LOGGER.info("Finished.  Cleaning up...");
+
+        LOGGER.info("Providing {} docs", providerQueue.size());
+
+        StreamsResultSet result =  new StreamsResultSet(providerQueue);
+        running.set(false);
+
+        LOGGER.info("Exiting");
+
+        return result;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+        Facebook client = getFacebookClient();
+
+        try {
+            ResponseList<Friend> friendResponseList = client.friends().getFriends();
+            Paging<Friend> friendPaging;
+            do {
+
+                for( Friend friend : friendResponseList ) {
+
+                    //client.rawAPI().callPostAPI();
+                    // add a subscription
+                }
+                friendPaging = friendResponseList.getPaging();
+                friendResponseList = client.fetchNext(friendPaging);
+            } while( friendPaging != null &&
+                    friendResponseList != null );
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(configuration.getOauth().getAppId())
+            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFeedPollingTask implements Runnable {
+
+        FacebookUserstreamProvider provider;
+        Facebook client;
+
+        private Set<Post> priorPollResult = Sets.newHashSet();
+
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
+            provider = facebookUserstreamProvider;
+        }
+
+        @Override
+        public void run() {
+            client = provider.getFacebookClient();
+            while (provider.isRunning()) {
+                try {
+                    ResponseList<Post> postResponseList = client.getHome();
+                    Set<Post> update = Sets.newHashSet(postResponseList);
+                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
+                    Set<Post> entrySet = Sets.difference(update, repeats);
+                    for (Post item : entrySet) {
+                        String json = DataObjectFactory.getRawJSON(item);
+                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                        try {
+                            lock.readLock().lock();
+                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                            countersCurrent.incrementAttempt();
+                        } finally {
+                            lock.readLock().unlock();
+                        }
+                    }
+                    priorPollResult = update;
+                    Thread.sleep(configuration.getPollIntervalMillis());
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserInformationProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserInformationProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserInformationProvider.java
new file mode 100644
index 0000000..fce5f22
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserInformationProvider.java
@@ -0,0 +1,299 @@
+/*
+ * 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.facebook.provider;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class FacebookUserInformationProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookUserInformationProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserInformationProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+
+    private Class klass;
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserInformationConfiguration getConfig()              { return facebookUserInformationConfiguration; }
+
+    public void setConfig(FacebookUserInformationConfiguration config)   { this.facebookUserInformationConfiguration = config; }
+
+    protected Iterator<String[]> idsBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookUserInformationProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config) {
+        this.facebookUserInformationConfiguration = config;
+    }
+
+    public FacebookUserInformationProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config, Class klass) {
+        this.facebookUserInformationConfiguration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        running.set(true);
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        Preconditions.checkArgument(idsBatches.hasNext());
+
+        LOGGER.info("readCurrent");
+
+        Facebook client = getFacebookClient();
+
+        try {
+            User me = client.users().getMe();
+            String json = mapper.writeValueAsString(me);
+            providerQueue.add(
+                new StreamsDatum(json, DateTime.now())
+            );
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+        if( idsBatches.hasNext()) {
+            while (idsBatches.hasNext()) {
+                try {
+                    List<User> userList = client.users().getUsers(idsBatches.next());
+                    for (User user : userList) {
+
+                        try {
+                            String json = mapper.writeValueAsString(user);
+                            providerQueue.add(
+                                    new StreamsDatum(json, DateTime.now())
+                            );
+                        } catch (JsonProcessingException e) {
+                            //                        e.printStackTrace();
+                        }
+                    }
+
+                } catch (FacebookException e) {
+                    e.printStackTrace();
+                }
+            }
+        } else {
+            try {
+                ResponseList<Friend> friendResponseList = client.friends().getFriends();
+                Paging<Friend> friendPaging;
+                do {
+
+                    for( Friend friend : friendResponseList ) {
+
+                        String json;
+                        try {
+                            json = mapper.writeValueAsString(friend);
+                            providerQueue.add(
+                                    new StreamsDatum(json)
+                            );
+                        } catch (JsonProcessingException e) {
+//                        e.printStackTrace();
+                        }
+                    }
+                    friendPaging = friendResponseList.getPaging();
+                    friendResponseList = client.fetchNext(friendPaging);
+                } while( friendPaging != null &&
+                        friendResponseList != null );
+            } catch (FacebookException e) {
+                e.printStackTrace();
+            }
+
+        }
+
+        LOGGER.info("Finished.  Cleaning up...");
+
+        LOGGER.info("Providing {} docs", providerQueue.size());
+
+        StreamsResultSet result =  new StreamsResultSet(providerQueue);
+        running.set(false);
+
+        LOGGER.info("Exiting");
+
+        return result;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppId());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getUserAccessToken());
+        Preconditions.checkNotNull(facebookUserInformationConfiguration.getInfo());
+
+        List<String> ids = new ArrayList<String>();
+        List<String[]> idsBatches = new ArrayList<String[]>();
+
+        for(String s : facebookUserInformationConfiguration.getInfo()) {
+            if(s != null)
+            {
+                ids.add(s);
+
+                if(ids.size() >= 100) {
+                    // add the batch
+                    idsBatches.add(ids.toArray(new String[ids.size()]));
+                    // reset the Ids
+                    ids = new ArrayList<String>();
+                }
+
+            }
+        }
+
+        if(ids.size() > 0)
+            idsBatches.add(ids.toArray(new String[ids.size()]));
+
+        this.idsBatches = idsBatches.iterator();
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(facebookUserInformationConfiguration.getOauth().getAppId())
+            .setOAuthAppSecret(facebookUserInformationConfiguration.getOauth().getAppSecret())
+            .setOAuthAccessToken(facebookUserInformationConfiguration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserstreamProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserstreamProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserstreamProvider.java
new file mode 100644
index 0000000..d4f30e2
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/org/apache/streams/facebook/provider/FacebookUserstreamProvider.java
@@ -0,0 +1,320 @@
+/*
+ * 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.facebook.provider;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.Post;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookUserstreamProvider implements StreamsProvider, Serializable {
+
+    public static final String STREAMS_ID = "FacebookUserstreamProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserstreamProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "read_stream";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig() {
+        return configuration;
+    }
+
+    public void setConfig(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    protected ListeningExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    protected Facebook client;
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookUserstreamProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookUserstreamProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
+        try {
+            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+
+        client = getFacebookClient();
+
+        if( configuration.getInfo() != null &&
+            configuration.getInfo().size() > 0 ) {
+            for( String id : configuration.getInfo()) {
+                executor.submit(new FacebookFeedPollingTask(this, id));
+            }
+            running.set(true);
+        } else {
+            try {
+                String id = client.getMe().getId();
+                executor.submit(new FacebookFeedPollingTask(this, id));
+                running.set(true);
+            } catch (FacebookException e) {
+                LOGGER.error(e.getMessage());
+                running.set(false);
+            }
+        }
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        StreamsResultSet current;
+
+        synchronized (FacebookUserstreamProvider.class) {
+            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
+            current.setCounter(new DatumStatusCounter());
+            current.getCounter().add(countersCurrent);
+            countersTotal.add(countersCurrent);
+            countersCurrent = new DatumStatusCounter();
+            providerQueue.clear();
+        }
+
+        return current;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet) providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+        client = getFacebookClient();
+
+        if( configuration.getInfo() != null &&
+            configuration.getInfo().size() > 0 ) {
+
+            List<String> ids = new ArrayList<String>();
+            List<String[]> idsBatches = new ArrayList<String[]>();
+
+            for (String s : configuration.getInfo()) {
+                if (s != null) {
+                    ids.add(s);
+
+                    if (ids.size() >= 100) {
+                        // add the batch
+                        idsBatches.add(ids.toArray(new String[ids.size()]));
+                        // reset the Ids
+                        ids = new ArrayList<String>();
+                    }
+
+                }
+            }
+        }
+    }
+
+    protected Facebook getFacebookClient() {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+                .setOAuthAppId(configuration.getOauth().getAppId())
+                .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+                .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+                .setOAuthPermissions(ALL_PERMISSIONS)
+                .setJSONStoreEnabled(true);
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFeedPollingTask implements Runnable {
+
+        FacebookUserstreamProvider provider;
+        Facebook client;
+        String id;
+
+        private Set<Post> priorPollResult = Sets.newHashSet();
+
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
+            this.provider = facebookUserstreamProvider;
+        }
+
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider, String id) {
+            this.provider = facebookUserstreamProvider;
+            this.client = provider.client;
+            this.id = id;
+        }
+        @Override
+        public void run() {
+            while (provider.isRunning()) {
+                ResponseList<Post> postResponseList;
+                try {
+                    postResponseList = client.getFeed(id);
+
+                    Set<Post> update = Sets.newHashSet(postResponseList);
+                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
+                    Set<Post> entrySet = Sets.difference(update, repeats);
+                    LOGGER.debug(this.id + " response: " + update.size() + " previous: " + repeats.size() + " new: " + entrySet.size());
+                    for (Post item : entrySet) {
+                        String json = DataObjectFactory.getRawJSON(item);
+                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                        try {
+                            lock.readLock().lock();
+                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                            countersCurrent.incrementAttempt();
+                        } finally {
+                            lock.readLock().unlock();
+                        }
+                    }
+                    priorPollResult = update;
+                } catch (Exception e) {
+                    e.printStackTrace();
+                } finally {
+                    try {
+                        Thread.sleep(configuration.getPollIntervalMillis());
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
deleted file mode 100644
index b4e5afb..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookConfiguration.json
+++ /dev/null
@@ -1,49 +0,0 @@
-{
-    "type": "object",
-    "$schema": "http://json-schema.org/draft-03/schema",
-    "id": "#",
-    "javaType" : "org.apache.streams.facebook.FacebookConfiguration",
-    "javaInterfaces": ["java.io.Serializable"],
-    "properties": {
-        "protocol": {
-            "type": "string",
-            "description": "The protocol"
-        },
-        "host": {
-            "type": "string",
-            "description": "The host"
-        },
-        "port": {
-            "type": "integer",
-            "description": "The port"
-        },
-        "version": {
-            "type": "string",
-            "description": "The version"
-        },
-        "endpoint": {
-            "type": "string",
-            "description": "The endpoint"
-        },
-        "oauth": {
-            "type": "object",
-            "dynamic": "true",
-            "javaType" : "org.apache.streams.facebook.FacebookOAuthConfiguration",
-            "javaInterfaces": ["java.io.Serializable"],
-            "properties": {
-                "appId": {
-                    "type": "string"
-                },
-                "appSecret": {
-                    "type": "string"
-                },
-                "appAccessToken": {
-                    "type": "string"
-                },
-                "userAccessToken": {
-                    "type": "string"
-                }
-            }
-        }
-   }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
deleted file mode 100644
index b351be9..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
+++ /dev/null
@@ -1,23 +0,0 @@
-{
-    "type": "object",
-    "$schema": "http://json-schema.org/draft-03/schema",
-    "id": "#",
-    "javaType" : "org.apache.streams.facebook.FacebookUserInformationConfiguration",
-    "javaInterfaces": ["java.io.Serializable"],
-    "extends": {"$ref":"FacebookConfiguration.json"},
-    "javaInterfaces": ["java.io.Serializable"],
-    "properties": {
-        "info": {
-            "type": "array",
-            "description": "A list of user IDs, indicating users of interest",
-            "items": {
-                "type": "string"
-            }
-        },
-        "pollIntervalMillis": {
-            "type": "integer",
-            "default" : "60000",
-            "description": "Polling interval in ms"
-        }
-     }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
deleted file mode 100644
index bcb2258..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
+++ /dev/null
@@ -1,22 +0,0 @@
-{
-    "type": "object",
-    "$schema": "http://json-schema.org/draft-03/schema",
-    "id": "#",
-    "javaType" : "org.apache.streams.facebook.FacebookUserstreamConfiguration",
-    "javaInterfaces": ["java.io.Serializable"],
-    "extends": {"$ref":"FacebookConfiguration.json"},
-    "properties": {
-        "info": {
-            "type": "array",
-            "description": "A list of user IDs, indicating users of interest",
-            "items": {
-                "type": "string"
-            }
-        },
-        "pollIntervalMillis": {
-            "type": "integer",
-            "default" : "60000",
-            "description": "Polling interval in ms"
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
deleted file mode 100644
index 23bcb08..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/graph/Post.json
+++ /dev/null
@@ -1,203 +0,0 @@
-{
-    "type": "object",
-    "$schema": "http://json-schema.org/draft-03/schema",
-    "id": "#",
-    "javaType": "org.apache.streams.facebook.Post",
-    "properties": {
-        "id": {
-            "type": "string"
-        },
-        "from": {
-            "type": "object",
-            "properties": {
-                "id": {
-                    "type": "string"
-                },
-                "name": {
-                    "type": "string"
-                }
-            }
-        },
-        "to": {
-            "type": "object",
-            "properties": {
-                "data": {
-                    "type": "array",
-                    "items": {
-                        "type": "object",
-                        "properties": {
-                            "id": {
-                                "type": "string"
-                            },
-                            "name": {
-                                "type": "string"
-                            }
-                        }
-                    }
-                }
-            }
-        },
-        "message": {
-            "type": "string"
-        },
-        "message_tags": {
-            "type": "object",
-            "properties": {
-                "data": {
-                    "type": "array",
-                    "items": {
-                        "type": "object",
-                        "properties": {
-                            "id": {
-                                "type": "string"
-                            },
-                            "name": {
-                                "type": "string"
-                            }
-                        }
-                    }
-                }
-            }
-        },
-        "picture": {
-            "type": "string"
-        },
-        "link": {
-            "type": "string"
-        },
-        "name": {
-            "type": "string"
-        },
-        "caption": {
-            "type": "string"
-        },
-        "description": {
-            "type": "string"
-        },
-        "source": {
-            "type": "string"
-        },
-        "icon": {
-            "type": "string"
-        },
-        "actions": {
-            "type": "array",
-            "items": {
-                "type": "object",
-                "properties": {
-                    "name": {
-                        "type": "string"
-                    },
-                    "link": {
-                        "type": "string"
-                    }
-                }
-            }
-        },
-        "comments": {
-            "type": "array",
-            "items": {
-                "type": "object",
-                "properties": {
-                    "id": {
-                        "type": "string"
-                    },
-                    "from": {
-                        "type": "string"
-                    },
-                    "message": {
-                        "type": "string"
-                    },
-                    "created_time": {
-                        "type": "string",
-                        "format" : "date-time"
-                    }
-                }
-            }
-        },
-        "likes": {
-            "type": "array",
-            "items": {
-                "type": "object",
-                "properties": {
-                    "name": {
-                        "type": "string"
-                    },
-                    "link": {
-                        "type": "string"
-                    }
-                }
-            }
-        },
-        "type": {
-            "type": "string"
-        },
-        "place": {
-            "type": "object",
-            "properties": {
-                "name": {
-                    "type": "string"
-                },
-                "id": {
-                    "type": "string"
-                }
-            }
-        },
-        "story": {
-            "type": "string"
-        },
-        "shares": {
-            "type": "int"
-        },
-        "object_id": {
-            "type": "int"
-        },
-        "application": {
-            "type": "object",
-            "properties": {
-                "name": {
-                    "type": "string"
-                },
-                "id": {
-                    "type": "string"
-                }
-            }
-        },
-        "created_time": {
-            "type": "string",
-            "format" : "date-time"
-        },
-        "updated_time": {
-            "type": "string",
-            "format" : "date-time"
-        },
-        "include_hidden": {
-            "type": "boolean"
-        },
-        "status_type": {
-            "type": "string"
-        },
-        "properties": {
-            "type": "array",
-            "items": {
-                "type": "object",
-                "properties": {
-                    "name": {
-                        "type": "string"
-                    },
-                    "text": {
-                        "type": "string"
-                    }
-                }
-            }
-        },
-        "privacy": {
-            "type": "object",
-            "properties": {
-                "value": {
-                    "type": "string"
-                }
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookConfiguration.json
new file mode 100644
index 0000000..b4e5afb
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookConfiguration.json
@@ -0,0 +1,49 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "properties": {
+        "protocol": {
+            "type": "string",
+            "description": "The protocol"
+        },
+        "host": {
+            "type": "string",
+            "description": "The host"
+        },
+        "port": {
+            "type": "integer",
+            "description": "The port"
+        },
+        "version": {
+            "type": "string",
+            "description": "The version"
+        },
+        "endpoint": {
+            "type": "string",
+            "description": "The endpoint"
+        },
+        "oauth": {
+            "type": "object",
+            "dynamic": "true",
+            "javaType" : "org.apache.streams.facebook.FacebookOAuthConfiguration",
+            "javaInterfaces": ["java.io.Serializable"],
+            "properties": {
+                "appId": {
+                    "type": "string"
+                },
+                "appSecret": {
+                    "type": "string"
+                },
+                "appAccessToken": {
+                    "type": "string"
+                },
+                "userAccessToken": {
+                    "type": "string"
+                }
+            }
+        }
+   }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserInformationConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserInformationConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserInformationConfiguration.json
new file mode 100644
index 0000000..b351be9
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserInformationConfiguration.json
@@ -0,0 +1,23 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookUserInformationConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "extends": {"$ref":"FacebookConfiguration.json"},
+    "javaInterfaces": ["java.io.Serializable"],
+    "properties": {
+        "info": {
+            "type": "array",
+            "description": "A list of user IDs, indicating users of interest",
+            "items": {
+                "type": "string"
+            }
+        },
+        "pollIntervalMillis": {
+            "type": "integer",
+            "default" : "60000",
+            "description": "Polling interval in ms"
+        }
+     }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserstreamConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserstreamConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserstreamConfiguration.json
new file mode 100644
index 0000000..bcb2258
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/FacebookUserstreamConfiguration.json
@@ -0,0 +1,22 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType" : "org.apache.streams.facebook.FacebookUserstreamConfiguration",
+    "javaInterfaces": ["java.io.Serializable"],
+    "extends": {"$ref":"FacebookConfiguration.json"},
+    "properties": {
+        "info": {
+            "type": "array",
+            "description": "A list of user IDs, indicating users of interest",
+            "items": {
+                "type": "string"
+            }
+        },
+        "pollIntervalMillis": {
+            "type": "integer",
+            "default" : "60000",
+            "description": "Polling interval in ms"
+        }
+    }
+}
\ No newline at end of file


[08/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/redditTest.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/redditTest.xml b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/redditTest.xml
index b2489fa..bd15166 100644
--- a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/redditTest.xml
+++ b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/redditTest.xml
@@ -1 +1 @@
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1088dt</id>		<published>2012-09-21T03:28:15Z</published>		<updated>2012-09-21T03:28:15Z</updated>		<title>Precious_Zest posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/atheism/comments/1088dt/a_classy_new_way_to_respond_to_stupid_fundie/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </gnip
 :rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1088dt</id>			<title>A classy new way to respond to stupid fundie claims (X-post from r/funny) </title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/atheism/comments/1088dt/a_classy_new_way_to_respond_to_stupid_fundie/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/ZphAO.png"/>			<link rel="preview" type="image/jpeg" href="http://a.thumbs.redditmedia.com/iL3sa_rwh2j-WA7H.jpg"/>			<gnip:statistics upVotes="220" downVotes="69"/>		</activity:object>		<author>			<name>Precious_Zest</name>			<uri>http://www.reddit.com/user/Precious_Zest</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:obje
 ct-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/Precious_Zest"/>			<id>http://www.reddit.com/user/Precious_Zest</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
\ No newline at end of file
+<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1088dt</id>		<published>2012-09-21T03:28:15Z</published>		<updated>2012-09-21T03:28:15Z</updated>		<title>Precious_Zest posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/atheism/comments/1088dt/a_classy_new_way_to_respond_to_stupid_fundie/"/>		<source>			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""   
          </gnip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1088dt</id>			<title>A classy new way to respond to stupid fundie claims (X-post from r/funny) </title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/atheism/comments/1088dt/a_classy_new_way_to_respond_to_stupid_fundie/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/ZphAO.png"/>			<link rel="preview" type="image/jpeg" href="http://a.thumbs.redditmedia.com/iL3sa_rwh2j-WA7H.jpg"/>			<gnip:statistics upVotes="220" downVotes="69"/>		</activity:object>		<author>			<name>Precious_Zest</name>			<uri>http://www.reddit.com/user/Precious_Zest</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person
 </activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/Precious_Zest"/>			<id>http://www.reddit.com/user/Precious_Zest</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
\ No newline at end of file


[09/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDCFlattened.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDCFlattened.xml b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDCFlattened.xml
index fb9af4f..f9e8486 100644
--- a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDCFlattened.xml
+++ b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDCFlattened.xml
@@ -1,100 +1,100 @@
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7fgd</id>		<published>2013-08-27T20:07:28Z</published>		<updated>2013-08-27T20:07:28Z</updated>		<title>TerriblePterodactyl posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/longrange/comments/1l7fgd/need_advice_selecting_between_two_rifles/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_135v09"/>			<title>Reddit - Keyword - Search (Recent) - "Remington"</title>			<updated>2013-08-28T19-54-43Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Remington"            </gnip:rule>		</source>		<service
 :provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7fgd</id>			<title>Need advice; Selecting Between Two Rifles</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;So, I have an opportunity to purchase a Remington 700 in .308, with a Vortex Viper 4-16X50 scope, Harris bipod and Standard Stock.  The other option is the Remington 700P in .308 with a Leupold Mark V 4.5-14x50 scope, Harris bipod and the HS stock.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The regular 700 with the Vortex is $200 cheaper.  This will be my first rifle, so I&amp;amp;#39;m looking to get in as low as possible, and still get a quality option.  Eventually, I&amp;amp;#39;d like to save up some money for the Savage 110 BA in .338, but I figured I
 &amp;amp;#39;d like the simplicity of the Remington, while still having options to upgrade the stock, get the action trued (if necessary) and even have a gunsmith work over to get the barrel truly free-floating.  I&amp;amp;#39;m not sure if any of that will be entirely necessary at first, as I&amp;amp;#39;m still learning the tricks.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Any advice on which one I should be leaning toward?&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/longrange/comments/1l7fgd/need_advice_selecting_between_two_rifles/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="11" downVotes="0"/>		</activity:object>		<author>			<name>TerriblePterodactyl</name>			<uri>http://www.reddit.com/user/TerriblePterodactyl</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel=
 "alternate" type="text/html" length="0" href="http://www.reddit.com/user/TerriblePterodactyl"/>			<id>http://www.reddit.com/user/TerriblePterodactyl</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="remington">"Remington"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l8rv5</id>		<published>2013-08-28T06:54:37Z</published>		<updated>2013-08-28T06:54:37Z</updated>		<title>darcgecko posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/KerbalSpaceProgram/comments/1l8rv5/tip_choosing_a_jet_engine_based_on_velocity/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_19h0y5"/>			<title>Reddit - Keyword - Search (Recent) - "Max Factor"</title>			<updated>2013-08-28T19-54-00Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Max Factor"            </gnip:rule>		</source>		<
 service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l8rv5</id>			<title>TIP: Choosing a jet engine based on velocity curves, efficiency, and thrust:mass ratios</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;As you may know, jet engines in this game have a property called a velocity curve. At max throttle, they will output a percentage of the listed max thrust (in the VAB) based on the velocity of the ship. This &amp;amp;#39;&amp;amp;#39;curve&amp;amp;#39;&amp;amp;#39; is defined at several points based on parameters in the part .cfg file, and should be considered when picking what jets to use on your ship.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;VELOCITY&amp;lt;/strong&amp;gt;&amp;l
 t;/p&amp;gt;&amp;lt;p&amp;gt;For the stock turbojet, the values are as follows:&amp;lt;/p&amp;gt;&amp;lt;blockquote&amp;gt;&amp;lt;p&amp;gt;velocityCurve&amp;lt;/p&amp;gt;&amp;lt;/blockquote&amp;gt;&amp;lt;pre&amp;gt;&amp;lt;code&amp;gt;    key = 0 0.5 0 0    key = 1000 1 0 0    key = 2000 0.5 0 0    key = 2400 0 0 0&amp;lt;/code&amp;gt;&amp;lt;/pre&amp;gt;&amp;lt;p&amp;gt;As you can see, the jet produces max thrust at 1000 m/s, and 0 thrust at 2400 m/s. This makes it useful for high speed cruising/ssto design. Furthermore, a quick test I ran gave me an output of 170 ish kn of thrust at 500 m/s, which is 75% of 225, and thus it seems that the scaling on this &amp;amp;#39;&amp;amp;#39;curve&amp;amp;#39;&amp;amp;#39; is in fact linear between the points designated.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The normal stock jet, meanwhile, looks like this:&amp;lt;/p&amp;gt;&amp;lt;blockquote&amp;gt;&amp;lt;p&amp;gt;velocityCurve&amp;lt;/p&amp;gt;&amp;lt;/blockquote&amp;gt;&amp;lt;pre&amp;gt;&a
 mp;lt;code&amp;gt;    key = 1000 0 0 0    key = 850 0.2 0 0    key = 0 1 0 0&amp;lt;/code&amp;gt;&amp;lt;/pre&amp;gt;&amp;lt;p&amp;gt;As you can see, this jet is designed for lower speed, and indeed has a far lower thrust:mass ratio at max thrust than the turbojet (150:1 compared to 187.5:1). However, it should theoretically make up for this by being more fuel efficient at high pressure (i.e. low altitude).&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;If we ignore efficiency for the moment, and simply graph the two in terms of thrust:mass ratio and velocity, it turns out that the equivalence point (past which turbojets are more powerful in all situations) happens at 239.4 m/s. 239 m/s is reaaaaaally slow, and so in cases where speed is the objective, the turbojets are going to be more effective than the standard jets.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;EFFICIENCY&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Of course, this doesn&amp;amp;#39;t factor in efficien
 cy. From the information in the VAB, it would appear that turbojets have an isp far lower than standard jets, as they range from only 800-1200. This data is misleading, however, as there is yet another &amp;amp;#39;curve&amp;amp;#39; parameter in the .cfg that controls isp relative to atmospheric pressure. &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The curve for the turbojet is as follows:&amp;lt;/p&amp;gt;&amp;lt;blockquote&amp;gt;&amp;lt;p&amp;gt;atmosphereCurve&amp;lt;/p&amp;gt;&amp;lt;/blockquote&amp;gt;&amp;lt;pre&amp;gt;&amp;lt;code&amp;gt;    key = 0 1200    key = 0.3 2500    key = 1 800&amp;lt;/code&amp;gt;&amp;lt;/pre&amp;gt;&amp;lt;p&amp;gt;and for the normal jet:&amp;lt;/p&amp;gt;&amp;lt;blockquote&amp;gt;&amp;lt;p&amp;gt;atmosphereCurve&amp;lt;/p&amp;gt;&amp;lt;/blockquote&amp;gt;&amp;lt;pre&amp;gt;&amp;lt;code&amp;gt;    key = 0 1000    key = 0.3 1800    key = 1 2000&amp;lt;/code&amp;gt;&amp;lt;/pre&amp;gt;&amp;lt;p&amp;gt;As you can see, while the normal jet gets less efficien
 t the higher you are, turbojets actually get &amp;lt;em&amp;gt;more&amp;lt;/em&amp;gt; efficient, until 0.3 atm (6020m, based on the equation for kerbin&amp;amp;#39;s atmo density on the wiki), before becoming less efficient as you get closer to vac. as they end up at 1200 isp in vac compared to the normal jet&amp;amp;#39;s 1000, it follows that after the you get high enough for the turbojet to become more efficient than the normal jet, it stays that way. This first intersection happens at 0.557 atm, which is only 2900m of altitude. Any higher than this, and turbojets are more efficient than normal jets.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;TO SUM UP&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;As you can see, in basically all situations, the turbojets are better than the normal jets. The same sort of analysis can be performed on mod parts, as all the info is in their part .cfg&amp;amp;#39;s. This also gives you an option if you really like a part&amp;
 amp;#39;s model or idea but think it&amp;amp;#39;s op - you can just modify the part .cfg until its stats line up with what you feel is balanced/realistic.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;TLDR: Basically, never use the normal jets. They&amp;amp;#39;re both less efficient and less powerful, unless you want to cruise around at below 240 m/s or below 2900m.&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/KerbalSpaceProgram/comments/1l8rv5/tip_choosing_a_jet_engine_based_on_velocity/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="5" downVotes="2"/>		</activity:object>		<author>			<name>darcgecko</name>			<uri>http://www.reddit.com/user/darcgecko</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user
 /darcgecko"/>			<id>http://www.reddit.com/user/darcgecko</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="max_factor">"Max Factor"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7v1p</id>		<published>2013-08-27T23:09:46Z</published>		<updated>2013-08-27T23:09:46Z</updated>		<title>Tylopodas posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/electronic_cigarette/comments/1l7v1p/first_mech_ordered_what_am_i_missing/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>			<title>Reddit - Keyword - Search (Recent) - "Panasonic"</title>			<updated>2013-08-28T19-47-07Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Panasonic"            </gnip:rule>		</source>		<service:pr
 ovider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7v1p</id>			<title>First mech ordered, what am I missing?</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Here&amp;amp;#39;s the list of stuff I ordered from various vendors:&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-Sigelei 8w&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-Sigelei kick clone&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-Nitecore I4&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-AW IMR 18490 x2&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-Panasonic NCR18650PD 18650 x2&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-IGO-W RDA (with monkeycan)&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;-RSST RBA&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Already have a multimeter, a couple feet of 2mm ekowool and some 32awg kanthal to get m
 e through until I can try out a few different style wicks/coils so I know what to order for mesh/silica/kanthal.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Anything I am missing that will greatly improve my first mech experience?&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/electronic_cigarette/comments/1l7v1p/first_mech_ordered_what_am_i_missing/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="7" downVotes="0"/>		</activity:object>		<author>			<name>Tylopodas</name>			<uri>http://www.reddit.com/user/Tylopodas</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/Tylopodas"/>			<id>http://www.reddit.com/user/Tylopodas</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag=
 "panasonic">"Panasonic"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l9kpj</id>		<published>2013-08-28T23:57:46Z</published>		<updated>2013-08-28T23:57:46Z</updated>		<title>SemiCharmedMike posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/videography/comments/1l9kpj/advice_dslr_vs_camcorders_for_wedding_videography/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>			<title>Reddit - Keyword - Search (Recent) - "Panasonic"</title>			<updated>2013-08-28T19-47-07Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Panasonic"            </gnip:rule>		</source>		<
 service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l9kpj</id>			<title>Advice: DSLR vs. Camcorders for Wedding Videography</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Hi everyone,&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;I just stumbled across this subreddit today and have lost an hour or so reading submissions. I am hoping you all will be able to help me out with a question I have.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;I am about to get into wedding videography and I&amp;amp;#39;m looking for the perfect (or as close to it as i can get to it) camera. For the last 5 years I have used camcorders for all of my productions (Panasonic Panasonic HVX200&amp;amp;#39;s to be exact) and I&amp;amp;#39;m familiar wi
 th them. Though lately I have been noticing that a lot of videographers are using DSLRs.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;In your opinion, which is better? I had myself talked into a Canon 7D until I started reading posts and now I&amp;amp;#39;m thinking about looking into Panasonic&amp;amp;#39;s G6.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;And then there is a voice in the back of my head that I should go with a camcorder, hah. See my dilemma?&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The main venue I will be producing the wedding videos at has outside ceremonies so I&amp;amp;#39;m not worried about low lighting there...but the reception hall can get dark so I may need a camera that can record well in low lighting.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;BTW - I have about $2000 budget for the camera and another $2000 for accessories/other equipment.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;P.S. Also planning on shooting short films with this camera.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Thoughts? Suggestions? Th
 anks in advance!&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/videography/comments/1l9kpj/advice_dslr_vs_camcorders_for_wedding_videography/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="2" downVotes="1"/>		</activity:object>		<author>			<name>SemiCharmedMike</name>			<uri>http://www.reddit.com/user/SemiCharmedMike</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/SemiCharmedMike"/>			<id>http://www.reddit.com/user/SemiCharmedMike</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="panasonic">"Panasonic"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7xtm</id>		<published>2013-08-27T23:44:57Z</published>		<updated>2013-08-27T23:44:57Z</updated>		<title>AnyelevNokova posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/RandomActsofMakeup/comments/1l7xtm/contest_im_hungry_go_make_me_something/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_sa7ex"/>			<title>Reddit - Keyword - Search (Recent) - "L'oreal" "loreal"</title>			<updated>2013-08-28T19-46-28Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"L'oreal" "loreal"            </gn
 ip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7xtm</id>			<title>[Contest] I'm Hungry; Go Make Me Something!</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Hey ladies (and men.) I&amp;amp;#39;ve never actually posted over this way before, though I&amp;amp;#39;ve browsed a bit. After digging through my battlestation, I realized that I have a lot of stuff that I used once or haven&amp;amp;#39;t even opened! I&amp;amp;#39;m not 100% sure if this kind of thing is even allowed or interesting to people, but I&amp;amp;#39;d like to give what I&amp;amp;#39;ve got away because, honestly, I&amp;amp;#39;m super lazy when it comes to returning things.&amp;lt;/p&amp;gt;&amp;lt;p&amp
 ;gt;SO. Because of the sheer quantity of stuff I have, I&amp;amp;#39;m going to be doing this Beauty Box style. I have three boxes which I have sorted based upon interest. That means there are &amp;lt;em&amp;gt;three&amp;lt;/em&amp;gt; people I am going to be sending these to. &amp;lt;/p&amp;gt;&amp;lt;hr/&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;What&amp;amp;#39;s in the Boxes?&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ul&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/yFM3p6H.jpg"&amp;gt;Yeah, Elves Work At Drugstores&amp;lt;/a&amp;gt;: ELF Contouring Blush &amp;amp;amp; Bronzer (lightly used), ELF High Definition Powder (sealed), ELF Eyebrow Kit (used once), ELF Eyelid Primer (lightly used), ELF Eye Primer &amp;amp;amp; Line Sealer (sealed), L&amp;amp;#39;Oreal Voluminous Mascara in Black-Brown (lightly used), Maybelline Color Tattoo Pure Pigments in Black Mystery and Buff &amp;amp;amp; Tuff (sealed), Maybelline Color Tattoo &amp;lt;em&amp;
 gt;creams&amp;lt;/em&amp;gt; in Pomegranate Punk and Bold Gold (sealed), Boscia Lavender Blotting Linens (opened and wearing on edges from purse travel; otherwise unused) &amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/ImSV4k6.jpg"&amp;gt;Let&amp;amp;#39;s Talk About Color&amp;lt;/a&amp;gt;: Revlon Photoready Perfecting Primer (sealed), Wet n Wild Eyeshadow Trios in I&amp;amp;#39;m Feeling Retro and Don&amp;amp;#39;t Steal My Thunder (sealed), Wet n Wild Color Icon Eyeshadow in Comfort Zone (sealed), Revlon Lip Butters in Creamsicle and Candy Apple (sealed), Revlon Lip Butter in Strawberry Shortcake (used once.) Not pictured but in the box: Boots No7 Beautiful Eyebrow Pencil in Brown (used once)&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/rp2JGWX.jpg"&amp;gt;Foxy Lady&amp;lt;/a&amp;gt;: Sephora Smokey Eye Brush (sealed), bareMinerals Marvelous Moxie Lipstick in Be
  Free (used once), Sephora Color Token Eyeshadow in Pink Casino (sealed), Tarte Amazonian Clay Full Coverage Airbrush Foundation in Light Beige (used once), Urban Decay Vitamin-Infused Complexion Prep Spray deluxe sample size (no seal, never used), Lady Gaga Fame deluxe sample size (no seal, never used)&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ul&amp;gt;&amp;lt;p&amp;gt;I&amp;amp;#39;ll also be tossing some random samples in the boxes as room allows.&amp;lt;/p&amp;gt;&amp;lt;hr/&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Rules! Rules! Rules!&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;US only. Sorry, but international shipping is eugh :(&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;I will &amp;lt;em&amp;gt;not&amp;lt;/em&amp;gt; be redistributing product within the boxes. What&amp;amp;#39;s in the box is in the box. If you win a box and don&amp;amp;#39;t want something in it, you&amp;amp;#39;re welcome
  to tell me and I&amp;amp;#39;ll remove the item. But no mix and match. If you don&amp;amp;#39;t like the boxes, don&amp;amp;#39;t enter.&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;One box per person. When you enter the contest, please list the three boxes in order of most wanted to least wanted. First winner gets their first pick. Second winner gets their first pick that isn&amp;amp;#39;t taken. Third winner gets the box that remains. If you are only interested in one or two boxes, you can choose to only list one or two, but you must explicitly specify that you &amp;lt;em&amp;gt;do not want&amp;lt;/em&amp;gt; the other(s). If you are selected as a winner, but the box(es) you listed were already taken, I&amp;amp;#39;ll move onto a runner-up who was less particular.&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&amp;gt;You must have a valid street address. No PO/FPO/APO/DPO.&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;li&amp;gt;&amp;lt;p&a
 mp;gt;No lurkers (I know, ironic) and/or new (&amp;amp;lt;30 day) accounts. I &amp;lt;em&amp;gt;will&amp;lt;/em&amp;gt; check your comment history.&amp;lt;/p&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;hr/&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Ok, what do I have to do?&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Make food. Take a picture of it. Post the picture alongside a description of what it is. Describe your dish to us. What does it taste like? How does it smell? What kind of texture does it have? Get creative here. Recipe is optional. I&amp;amp;#39;m not necessarily looking for the most tasty &amp;lt;em&amp;gt;looking&amp;lt;/em&amp;gt; dish; I&amp;amp;#39;m looking for the combination of picture and dish that makes me go OMG GIVE IT TO ME NOW. Yes: this is a very subjective contest. I have no criteria for meal or ingredients; however, it must be a singular dish. You may garnish, but I&amp;amp;#39;m not looking for Steak + Potatoes + Beans + Wine 
 + Candles. Show me the steak on a plate, and then tell me about it. Seriously. I&amp;amp;#39;m hungry!&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;The deadline for entries will be Wednesday, September 4th at 11:59 PM. I will announce winners by Friday morning.&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;em&amp;gt;Edit: Changed the deadline so people had more time to plan and cook.&amp;lt;/em&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/RandomActsofMakeup/comments/1l7xtm/contest_im_hungry_go_make_me_something/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="12" downVotes="2"/>		</activity:object>		<author>			<name>AnyelevNokova</name>			<uri>http://www.reddit.com/user/AnyelevNokova</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<
 link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/AnyelevNokova"/>			<id>http://www.reddit.com/user/AnyelevNokova</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="l'oreal">"L'oreal" "loreal"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l8bju</id>		<published>2013-08-28T02:45:46Z</published>		<updated>2013-08-28T02:45:46Z</updated>		<title>Badger4You posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/headphones/comments/1l8bju/recommendations_for_a_recreational_gaming_pc/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_156seg"/>			<title>Reddit - Keyword - Search (Recent) - "Max Factor"</title>			<updated>2013-08-28T19-39-36Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Max Factor"            </gnip:rule>		</source>		<servic
 e:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l8bju</id>			<title>Recommendations for a Recreational Gaming PC Headphones/Headset</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;[Price Range $80 to $120 max]  I&amp;amp;#39;m building my first PC and am looking for a nice quality pair of headphones to go with a modmic &amp;lt;a href="http://www.modmic.com/"&amp;gt;http://www.modmic.com/&amp;lt;/a&amp;gt; or just a headset but from what I read that&amp;amp;#39;s not the best idea. I really want surround sound and some good bass for a realistic feel, but I&amp;amp;#39;d still to have some sense of direction ie footsteps in call of duty, but overall sound quality is my priority for playing Mmorpgs or ga
 mes like Skyrim. I&amp;amp;#39;m pretty dead set on having a closed over ear style. I&amp;amp;#39;m not positive what kind of connections I have with my Asus motherboard &amp;lt;a href="http://www.newegg.com/Product/Product.aspx?Item=N82E16813131872&amp;amp;amp;nm_mc=EMC-IGNEFL082713&amp;amp;amp;cm_mmc=EMC-IGNEFL082713-_-EMC-082713-Index-_-AMDMotherboards-_-13131872-L0D"&amp;gt;http://www.newegg.com/Product/Product.aspx?Item=N82E16813131872&amp;amp;amp;nm_mc=EMC-IGNEFL082713&amp;amp;amp;cm_mmc=EMC-IGNEFL082713-_-EMC-082713-Index-_-AMDMotherboards-_-13131872-L0D&amp;lt;/a&amp;gt;. I do not have a sound card, so if you&amp;amp;#39;d recommend one please factor that into the price as well as an amp. Thanks!!&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/headphones/comments/1l8bju/recommendations_for_a_recreational_gaming_pc/"/>			<gnip:statistics upVotes="2" downVotes="0"/>		</activity:
 object>		<author>			<name>Badger4You</name>			<uri>http://www.reddit.com/user/Badger4You</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/Badger4You"/>			<id>http://www.reddit.com/user/Badger4You</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="max_factor">"Max Factor"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7vbs</id>		<published>2013-08-27T23:13:25Z</published>		<updated>2013-08-27T23:13:25Z</updated>		<title>flip69 posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SanDiegoClassifieds/comments/1l7vbs/anyone_need_a_large_hepa_filter_ive_got_a_two/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100"/>			<title>Reddit - Keyword - Search (Recent) - "lysol"</title>			<updated>2013-08-28T19-35-40Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"lysol"            </gnip:rule>		</source>		<service:provider>			<name>Reddit</nam
 e>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7vbs</id>			<title>Anyone need a LARGE HEPA Filter I've got a two person filter with flow hood for sale!!! it comes with a pre-filter addition.</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;You read that right... perfect for lab work and all the other things a person might want to do...  it&amp;amp;#39;s a dual seat/person HEPA filter with clear flow hood that&amp;amp;#39;s perfect for those people that want to work in a sterile environment. &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Same model as being sold for over $1,600.00 but I&amp;amp;#39;ll let it go for less than 1/3 that it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it can&amp;amp;#39;t be beat.&amp
 ;lt;/p&amp;gt;&amp;lt;p&amp;gt;24 x 36  scanned 99.99% efficient micron filter, maximizing your sterile work area. Comes with a prefilter assembly, a powerful 1⁄2 HP blower, and grounded plug and cord. Truly the &amp;amp;quot;Cadillac&amp;amp;quot; of Laminar Flow Hoods! &amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SanDiegoClassifieds/comments/1l7vbs/anyone_need_a_large_hepa_filter_ive_got_a_two/"/>			<gnip:statistics upVotes="1" downVotes="0"/>		</activity:object>		<author>			<name>flip69</name>			<uri>http://www.reddit.com/user/flip69</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/flip69"/>			<id>http://www.reddit.com/user/flip69</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Lys
 ol">"lysol"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l9ek9</id>		<published>2013-08-28T22:40:15Z</published>		<updated>2013-08-28T22:40:15Z</updated>		<title>beautiful_sausage posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Fapucational/comments/1l9ek9/by_1890_many_variations_of_soap_were_offered_with/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22colgate%22&amp;limit=100&amp;after=t3_1cfq0r"/>			<title>Reddit - Keyword - Search (Recent) - "colgate"</title>			<updated>2013-08-28T19-34-16Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"colgate"            </gnip:rule>		</source>		<ser
 vice:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l9ek9</id>			<title>By 1890 many variations of soap were offered, with the five major companies being, Colgate, Morse Taylor, Albert, Pears, and Bailey.</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Fapucational/comments/1l9ek9/by_1890_many_variations_of_soap_were_offered_with/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/RItI7sq.jpg"/>			<link rel="preview" type="image/jpeg" href="nsfw"/>			<gnip:statistics upVotes="11" downVotes="0"/>		</activity:object>		<author>			<name>beautiful_sausage</name>			<uri>http://www.reddit.com/user/beautiful_sausage</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity
 :object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/beautiful_sausage"/>			<id>http://www.reddit.com/user/beautiful_sausage</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Colgate">"colgate"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l9u33</id>		<published>2013-08-29T01:49:52Z</published>		<updated>2013-08-29T01:49:52Z</updated>		<title>SeleneNyx posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l9u33/been_sick_for_days_but_gifts_from_loreal_perked/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_1a5xa8"/>			<title>Reddit - Keyword - Search (Recent) - "L'oreal" "loreal"</title>			<updated>2013-08-28T19-32-14Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"L'oreal" "loreal"            <
 /gnip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l9u33</id>			<title>Been sick for days, but gifts from L'Oreal perked me up! (info in comments)</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l9u33/been_sick_for_days_but_gifts_from_loreal_perked/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/2K18XEq.jpg"/>			<link rel="preview" type="image/jpeg" href="http://b.thumbs.redditmedia.com/TsOwJ_IrneE_d3NO.jpg"/>			<gnip:statistics upVotes="9" downVotes="2"/>		</activity:object>		<author>			<name>SeleneNyx</name>			<uri>http://www.reddit.com/user/SeleneNyx</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activ
 ity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/SeleneNyx"/>			<id>http://www.reddit.com/user/SeleneNyx</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="l'oreal">"L'oreal" "loreal"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l9g7d</id>		<published>2013-08-28T23:02:22Z</published>		<updated>2013-08-28T23:02:22Z</updated>		<title>cucumm2 posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/amazonsdeal/comments/1l9g7d/amazon_lysol_disinfecting_wipes_upto_35_off_plus/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100&amp;after=t3_10zvhy"/>			<title>Reddit - Keyword - Search (Recent) - "lysol"</title>			<updated>2013-08-28T19-28-30Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"lysol"            </gnip:rule>		</source>		<service:provider>			<
 name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l9g7d</id>			<title>Amazon: Lysol Disinfecting Wipes upto 35% off plus buy 4 get 1 free</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/amazonsdeal/comments/1l9g7d/amazon_lysol_disinfecting_wipes_upto_35_off_plus/"/>			<link rel="enclosure" type="text/html" href="http://www.usmomdeal.com/amazon-lysol-disinfecting-wipes-upto-35-plus-buy-4-get-1-free/"/>			<link rel="preview" type="image/jpeg" href="http://c.thumbs.redditmedia.com/dam2sY5sFiayvl6J.jpg"/>			<gnip:statistics upVotes="1" downVotes="0"/>		</activity:object>		<author>			<name>cucumm2</name>			<uri>http://www.reddit.com/user/cucumm2</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:
 object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/cucumm2"/>			<id>http://www.reddit.com/user/cucumm2</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Lysol">"lysol"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7ym9</id>		<published>2013-08-27T23:55:46Z</published>		<updated>2013-08-27T23:55:46Z</updated>		<title>vash3g posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shotguns/comments/1l7ym9/remington_1100_stock/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_1fk3mo"/>			<title>Reddit - Keyword - Search (Recent) - "Remington"</title>			<updated>2013-08-28T19-25-58Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Remington"            </gnip:rule>		</source>		<service:provider>			<name>Reddit</name>		
 	<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7ym9</id>			<title>Remington 1100 stock</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;I am purchasing a Remington 1100 20 Guage from gunbroker at a great price. The seller called me last night to say that the stock was not adult sized and short. I&amp;amp;#39;m assuming it might be a youth stock. Where is the best place outside of Remington direct to get a nice monte carlo checkered stock to replace it?&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shotguns/comments/1l7ym9/remington_1100_stock/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="7" downV
 otes="1"/>		</activity:object>		<author>			<name>vash3g</name>			<uri>http://www.reddit.com/user/vash3g</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/vash3g"/>			<id>http://www.reddit.com/user/vash3g</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="remington">"Remington"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1la0ec</id>		<published>2013-08-29T03:03:30Z</published>		<updated>2013-08-29T03:03:30Z</updated>		<title>skincarerevamp posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SkincareAddiction/comments/1la0ec/need_help_finding_a_new_skincare_routine/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Aveeno%22&amp;limit=100"/>			<title>Reddit - Keyword - Search (Recent) - "Aveeno"</title>			<updated>2013-08-28T19-24-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Aveeno"            </gnip:rule>		</source>		<service:provider>			<name>Reddit<
 /name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1la0ec</id>			<title>Need help finding a new skincare routine.</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;I recently found this subreddit, and since then have discovered the many errors of my skincare ways. I am a 26(F) who has been using Proactiv for about 3-4 years now. I know that there are less expensive products I can use that would be better on my skin.  &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Skin: I have somewhat oily skin, my forehead can get shiny. The pores on my nose and chin clog easily but my cheeks are always really smooth and clear. My biggest annoyance is my nose, which has been red and tends to flake since I was a teenager. I will get the occasional breakout, almo
 st always hormonal. As long as I use Proactiv religiously, my skin stays overall pretty decent looking.  Also I have dark circles/uneven skin tone under my eyes and have tried many products without finding one I really love.  &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Routine: I think I have overdone it with my routine in the past. I have recently stopped exfoliating as often, but I will list my previous routine.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;AM – St. Ives scrub, Proactiv facewash (with or without Clarisonic), Proactiv toner over entire face, Proactiv repairing treatment, Simple protecting light moisturizer SPF 15, Clean &amp;amp;amp; Clear Advantage Spot Treatment, eye cream/serum (either Philosophy Eye Believe or Genifique Yeux Light-Pearl)&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;PM – Ponds makeup removing wipes, Proactiv facewash (with or without Clarisonic), Proactiv toner over entire face, Proactiv clarifying night cream, Proactiv repairing treatment, Simple protecting light moisturi
 zer SPF 15, eye cream/serum&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Also once or twice a week I may use Proactiv refining mask, Proactiv blackhead dissolving gel, or   Proactiv glycolic acid peel&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Over the past few weeks I have stopped using the St. Ives Scrub and switched to an Aveeno scrub, which I can tell is a lot more gentle on my face. I only use the scrub now a couple times a week and my Clarisonic every few days at night.  &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;So basically I’m looking for advice because this routine is quite tedious/expensive and it works decently but not great.  Thanks!&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SkincareAddiction/comments/1la0ec/need_help_finding_a_new_skincare_routine/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="1" downVotes="0"/>		</activity:object>		<author>			<name>ski
 ncarerevamp</name>			<uri>http://www.reddit.com/user/skincarerevamp</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/skincarerevamp"/>			<id>http://www.reddit.com/user/skincarerevamp</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="aveeno">"Aveeno"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l85hv</id>		<published>2013-08-28T01:24:53Z</published>		<updated>2013-08-28T01:24:53Z</updated>		<title>justateburrito posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shave_Bazaar/comments/1l85hv/fs_den_clearance_fatboy_rocket_simpson/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22schick%22&amp;limit=100&amp;after=t3_13oj1g"/>			<title>Reddit - Keyword - Search (Recent) - "schick"</title>			<updated>2013-08-28T19-22-35Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"schick"            </gnip:rule>		</source>		<service:provider>			
 <name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l85hv</id>			<title>FS: Den Clearance, Fatboy, Rocket, Simpson, Penahaligon's, Bakelite, Soaps, Razors</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Gentlemen(and ladies), I&amp;amp;#39;m looking to clear out some items that I enjoy having around, but don&amp;amp;#39;t show enough love to. I hope you can find a place in your den for them. &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;I will only ship CONUS and accept payments though Paypal. If you see a comment below that a PM was sent about a specific item, it does not mean a sale was made until I mark it sold. I may need a couple of days to get these items shipped out, but will do my best to be speedy. Items will ship USPS t
 o your confirmed Paypal address only. If you&amp;amp;#39;re interested in multiple items I will knock something off total price, depending on the items.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The only items I would be willing to make a trade for would be a Mint Merkur 37G(the gold slant) or a minty Aristocrat #66.  &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Razors&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Gillette Fatboy, date code G2. This is nice and shiny. I touched up the numbers with enamel paint when I first received it. This comes in the case which seems to be in pretty good shape as well. Also included is the blade case which appears to have a few blades in it, not sure how many. Price $100&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/o8CWQDS.jpg"&amp;gt;http://i.imgur.com/o8CWQDS.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href=
 "http://i.imgur.com/soiDjtF.jpg"&amp;gt;http://i.imgur.com/soiDjtF.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/E6lETDo.jpg"&amp;gt;http://i.imgur.com/E6lETDo.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/2Wk1VmL.jpg"&amp;gt;http://i.imgur.com/2Wk1VmL.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Gillette British Rocket HD, with the case. Both are in nice condition, the case isn&amp;amp;#39;t cracked and the razor is shiny. I purchased this on the BST in March and haven&amp;amp;#39;t used it much. Price $ 85&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/sIYtnK5.jpg"&amp;gt;http://i.imgur.com/sIYtnK5.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/rtcTROt.jpg"&amp;gt;http://i.imgur.com/rtcTROt.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&a
 mp;gt;&amp;lt;a href="http://i.imgur.com/cLXpwH3.jpg"&amp;gt;http://i.imgur.com/cLXpwH3.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/RvUP4ro.jpg"&amp;gt;http://i.imgur.com/RvUP4ro.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;NOS Bakelite Slant w/case. This is the one that was being sold a few months back. It was my backup and I never used it. I am not sure I want to let it go, but I will for the right price. Listing this one at $100&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/AUhHfLj.jpg"&amp;gt;http://i.imgur.com/AUhHfLj.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/eq5anFc.jpg"&amp;gt;http://i.imgur.com/eq5anFc.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Schick Injector - I believe this is a G3, it is all metal. I purchased it in it&amp;amp;#39;s cu
 rrent condition with the intent of cleaning it up but haven&amp;amp;#39;t gotten too it. There are no dents in it, although the head has a slight wiggle where the arrow is pointing in the picture. I&amp;amp;#39;m sure it&amp;amp;#39;s easily remedied. The plating is not good, but I do not see these often and it would make a nice piece for a re-plate. Price $25&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/5DhODtQ.jpg"&amp;gt;http://i.imgur.com/5DhODtQ.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/VZ5iayO.jpg"&amp;gt;http://i.imgur.com/VZ5iayO.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/291YUGv.jpg"&amp;gt;http://i.imgur.com/291YUGv.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Razor lot. I bought these razors with the intention of cleaning them up and using them but just haven&amp;am
 p;#39;t had time. The lot includes a slim adjustable(L1), Ericsson open comb, and a Christy razor. the adjustable seems to open/close and adjust. If you want to clean these up and see if they&amp;amp;#39;re in useable condition, give it a shot. Price $30&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/KgrqSfO.jpg"&amp;gt;http://i.imgur.com/KgrqSfO.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/fxipANp.jpg"&amp;gt;http://i.imgur.com/fxipANp.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Brushes&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Simspon Duke 2 in Best. You know what it is. It&amp;amp;#39;s in good condition with the sticker still attached and lettering nice and clear. Comes with the original box. Price $65&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&
 amp;gt;&amp;lt;a href="http://i.imgur.com/kuuUd6t.jpg"&amp;gt;http://i.imgur.com/kuuUd6t.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/cSRZ5vN.jpg"&amp;gt;http://i.imgur.com/cSRZ5vN.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;SOLD &amp;lt;del&amp;gt;Whipped Dog Silvertip 24mm in Black Resin. Bought this because I just wanted to try a Whipped Dog Silvertip since so many people talk about them. I used it very briefly and purchased a 30mm from WD so this just sits on the shelf. It sheds every now and then, but not terribly. Price $20&amp;lt;/del&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/ir4OYUB.jpg"&amp;gt;http://i.imgur.com/ir4OYUB.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/W1nKbd0.jpg"&amp;gt;http://i.imgur.com/W1nKbd0.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&am
 p;gt;&amp;lt;strong&amp;gt;Soaps&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;SOLD &amp;lt;del&amp;gt;New unused Tabac in the bowl and new/unused Razorock Cacao(discontinued) selling these as package, not splitting it up. Price $30&amp;lt;/del&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pic&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/2eoxSmM.jpg"&amp;gt;http://i.imgur.com/2eoxSmM.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;D. R. Harris Arlington Soap in Mahogany bowl and Aftershave Milk. This stuff is great, I simply have too many soaps. There is at least 90-95% left of these, selling together with the original boxes. Price $50&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/rwuB4wv.jpg"&amp;gt;http://i.imgur.com/rwuB4wv.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.
 imgur.com/jx6UqU4.jpg"&amp;gt;http://i.imgur.com/jx6UqU4.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/43z4GRc.jpg"&amp;gt;http://i.imgur.com/43z4GRc.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;SOLD &amp;lt;del&amp;gt;Vintage Penhaligon&amp;amp;#39;s English Fern. This is from Bullgoose. I used a vegetable peeler on the side to fit it in the oak bowl, the &amp;amp;quot;peelings&amp;amp;quot; are underneath the puck in the bowl. I used this somewhere between 3-5 times. The lettering is still visible on the top of the puck. I will also include the remainder or a Penhaligon&amp;amp;#39;s English Fern Cologne decant, there is approx 2ml left. The oak bowl was purchased separately and I will sell it with or without the bowl. Price with the bowl $60 without the bowl $55&amp;lt;/del&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.co
 m/YnwFZwR.jpg"&amp;gt;http://i.imgur.com/YnwFZwR.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/VbF1RCy.jpg"&amp;gt;http://i.imgur.com/VbF1RCy.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Soap Lot. I enjoy all of these, but simply can&amp;amp;#39;t shave often enough to go through them all. Razorock Chianti Lavender, this is a soft soap, I do not believe he currently makes it anymore at least 80% left. Mama Bear&amp;amp;#39;s Indian Tobacco Flower &amp;amp;amp; Ye Olde Barbershoppe, both at least 90% left. RazoRock Mudder Focker (limited edition) at least 90% left. Valobra Menthol Croap, used 3-5 times and formed into a Boston Market side dish bowl. Also including the Proraso White Balm shown in the pictures, only used 2-3 times. Price $50&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/wrRkEjd.jpg"&amp;gt;http://i.imgur.com/wrRkEj
 d.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/IsrUQJM.jpg"&amp;gt;http://i.imgur.com/IsrUQJM.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Other Items&amp;lt;/strong&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Slim Zippo Lighter - I haven&amp;amp;#39;t used this, I purchased it in it&amp;amp;#39;s current condition. The flint produces a spark. There is a tiny hole on one side as shown in the picture although the picture makes it seem larger than it actually is. Price $10 sold with another item only.&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/RVm6RhS.jpg"&amp;gt;http://i.imgur.com/RVm6RhS.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/fbbQ7FR.jpg"&amp;gt;http://i.imgur.com/fbbQ7FR.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Custom B
 allpoint Pen by Elite Razor. This is Tourmaline Quartz with platinum fittings. Very nice piece with great weight to it. I paid $85. Asking $60&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/rhcgVUa.jpg"&amp;gt;http://i.imgur.com/rhcgVUa.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/6Ca67w5.jpg"&amp;gt;http://i.imgur.com/6Ca67w5.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;ol&amp;gt;&amp;lt;li&amp;gt;Whipped Dog Poor Man&amp;amp;#39;s Strop Kit. I used this once before deciding straights weren&amp;amp;#39;t for me. This strop isn&amp;amp;#39;t perfect, but it&amp;amp;#39;s useable for a beginner. There is a tiny nick shown in the pictures, not sure if this would make a difference, it&amp;amp;#39;s how I received it. Price $20&amp;lt;/li&amp;gt;&amp;lt;/ol&amp;gt;&amp;lt;p&amp;gt;&amp;amp;quot;The Poor Man strop kit contains a 2&amp;amp;quot; x 26&amp;amp;quo
 t; leather strop and a balsa strop to which abrasives have been added. It also includes a hook for wall mounting, and a small packet of Neatsfoot oil for treating the leather strop. The leather strop can also be looped around a towel rod as shown in the pictures.The balsa wood has been treated on one side with 0.3 micron chromium oxide (green) and with 0.1 micron iron oxide (red) on the other side. Additional supply of these abrasives are included.&amp;amp;quot;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Pics&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/flDEJyv.jpg"&amp;gt;http://i.imgur.com/flDEJyv.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://i.imgur.com/8ft7E62.jpg"&amp;gt;http://i.imgur.com/8ft7E62.jpg&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shave_Bazaar/comments/1l85hv/fs_den_clearance_fatboy_rocket_simpson/
 "/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="9" downVotes="2"/>		</activity:object>		<author>			<name>justateburrito</name>			<uri>http://www.reddit.com/user/justateburrito</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/justateburrito"/>			<id>http://www.reddit.com/user/justateburrito</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Schick ">"schick"</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7tb6</id>		<published>2013-08-27T22:49:12Z</published>		<updated>2013-08-27T22:49:12Z</updated>		<title>oh_rio_rio posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l7tb6/need_help_any_opinions_about_loreal_cosmetics/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22L%27oreal%22%20%22loreal%22%20%29&amp;limit=100&amp;after=t3_107krl"/>			<title>Reddit - Keyword - Search (Recent) - ("L'oreal" "loreal" )</title>			<updated>2013-08-28T18-40-15Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("L'oreal" "loreal" 
 )            </gnip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7tb6</id>			<title>Need help! Any opinions about L'Oreal Cosmetics? Event coming up...</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Hi you guys!&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Lucky me, I have been chosen as an artist for the event I&amp;amp;#39;ve linked to below. It&amp;amp;#39;s the Cosmo magazine and L&amp;amp;#39;Oreal Cosmetic Campus Tour! I&amp;amp;#39;m really excited, but a little concerned. I&amp;amp;#39;m a stylist with a medium-sized Aveda salon, and I&amp;amp;#39;m more than comfortable with professional application and session work. I am not, however, familiar AT ALL with L&amp;amp;#39;Ore
 al makeup. The only training I will be receiving is the consumer website and about two hours the morning of the event! Does anyone have any opinions, feedback, knowledge of these products?? I&amp;amp;#39;d love any help??&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;a href="http://www.cosmopolitan.com/hairstyles-beauty/skin-care-makeup/cosmo-campus-tour"&amp;gt;http://www.cosmopolitan.com/hairstyles-beauty/skin-care-makeup/cosmo-campus-tour&amp;lt;/a&amp;gt;&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l7tb6/need_help_any_opinions_about_loreal_cosmetics/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="3" downVotes="1"/>		</activity:object>		<author>			<name>oh_rio_rio</name>			<uri>http://www.reddit.com/user/oh_rio_rio</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</acti
 vity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/oh_rio_rio"/>			<id>http://www.reddit.com/user/oh_rio_rio</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="l'oreal">("L'oreal" "loreal" )</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l7btz</id>		<published>2013-08-27T19:25:31Z</published>		<updated>2013-08-27T19:25:31Z</updated>		<title>NickPressnall posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EarthPorn/comments/1l7btz/cutthroat_pass_pacific_crest_trail_wa_2560x1920/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22crest%22%20%29%20-%20%28%22pacific%20crest%22%20%29&amp;limit=100&amp;after=t3_10m52h"/>			<title>Reddit - Keyword - Search (Recent) - ("crest" ) - ("pacific crest" )</title>			<updated>2013-08-28T18-03-31Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas
 /2010">("crest" ) - ("pacific crest" )            </gnip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l7btz</id>			<title>Cutthroat Pass, Pacific Crest Trail, WA [2560x1920]</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EarthPorn/comments/1l7btz/cutthroat_pass_pacific_crest_trail_wa_2560x1920/"/>			<link rel="enclosure" type="text/html" href="http://imgur.com/ndkVEBK"/>			<link rel="preview" type="image/jpeg" href="http://a.thumbs.redditmedia.com/tdar1BPz9bv3JmGx.jpg"/>			<gnip:statistics upVotes="8" downVotes="1"/>		</activity:object>		<author>			<name>NickPressnall</name>			<uri>http://www.reddit.com/user/NickPressnall</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/s
 chema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/NickPressnall"/>			<id>http://www.reddit.com/user/NickPressnall</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Crest">("crest" ) - ("pacific crest" )</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l842y</id>		<published>2013-08-28T01:05:58Z</published>		<updated>2013-08-28T01:05:58Z</updated>		<title>mittens2248 posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MLPLounge/comments/1l842y/colgate_beadsprite_courtesy_of_opti/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22colgate%22%20%29&amp;limit=100&amp;after=t3_j7xv0"/>			<title>Reddit - Keyword - Search (Recent) - ("colgate" )</title>			<updated>2013-08-28T18-03-27Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("colgate" )            </gnip:rule>		</source>		<service:prov
 ider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l842y</id>			<title>Colgate Beadsprite, courtesy of Opti</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MLPLounge/comments/1l842y/colgate_beadsprite_courtesy_of_opti/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/FbGICCJ.jpg"/>			<gnip:statistics upVotes="45" downVotes="3"/>		</activity:object>		<author>			<name>mittens2248</name>			<uri>http://www.reddit.com/user/mittens2248</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/mittens2248"/>			<id>http://www.reddit.com/user/mittens2248</id>		</activity:actor>		<gni
 p:matching_rules>			<gnip:matching_rule rel="source" tag="Colgate">("colgate" )</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1l8qmr</id>		<published>2013-08-28T06:29:57Z</published>		<updated>2013-08-28T06:29:57Z</updated>		<title>dejoblue posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EQNext/comments/1l8qmr/raids_without_loot/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22Max%20Factor%22%20%29&amp;limit=100&amp;after=t3_zkcy0"/>			<title>Reddit - Keyword - Search (Recent) - ("Max Factor" )</title>			<updated>2013-08-28T18-01-30Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("Max Factor" )            </gnip:rule>		</source>		<service:provider>			<nam
 e>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1l8qmr</id>			<title>Raids Without Loot</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;I was thinking about not having levels and about what can be a motivating factor. As usual I resorted to my end game conclusion. Once you get to max level there is no more leveling, no more XP, what drives players? Character progression of some sort, usually in the form of a gear treadmill.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;What can we do that is different than the traditional loot drop from random raid boss X?&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;I thought back to my Nintendo upbringing to Mario and Zelda and Final Fantasy.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Saving the princess, making the
  triforce, defeating all the bosses and saving the world.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The princess can be kidnapped over and over, many magical items can be forged and the world will need saving many times over.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;This is what could be the driving force for EQN. Instead of loot drops perhaps a boss is guarding a magical cavern filled with rare ore to be mined.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;Maybe the boss has captured an Orc princess and when we save her we learn the Orc language which helps us discover some hidden treasure because we can read the orcish map we found at the beginning of the quest to save the princess.&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;The ubiquitous forging of a magical blade could be the reason to defeat a raid boss, plunge your already forged weapon into the dragon&amp;amp;#39;s steaming dead body and seal in it&amp;amp;#39;s magical powers to infuse your weapon with might and power!&amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;What o
 ther non loot scenarios can you think of guys?!&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EQNext/comments/1l8qmr/raids_without_loot/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="13" downVotes="5"/>		</activity:object>		<author>			<name>dejoblue</name>			<uri>http://www.reddit.com/user/dejoblue</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/dejoblue"/>			<id>http://www.reddit.com/user/dejoblue</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="max_factor">("Max Factor" )</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1brddb</id>		<published>2013-04-05T22:29:01Z</published>		<updated>2013-04-05T22:29:01Z</updated>		<title>lemon_zest posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/realasians/comments/1brddb/i_guess_ill_go_outside_after_this_sunshine_ftw_d/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </
 gnip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1brddb</id>			<title>I guess I'll go outside after this; sunshine ftw :D</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/realasians/comments/1brddb/i_guess_ill_go_outside_after_this_sunshine_ftw_d/"/>			<link rel="enclosure" type="text/html" href="http://imgur.com/oyhmkzU"/>			<link rel="preview" type="image/jpeg" href="http://b.thumbs.redditmedia.com/TykUrjG05wYU0wiy.jpg"/>			<gnip:statistics upVotes="71" downVotes="68"/>		</activity:object>		<author>			<name>lemon_zest</name>			<uri>http://www.reddit.com/user/lemon_zest</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="a
 lternate" type="text/html" length="0" href="http://www.reddit.com/user/lemon_zest"/>			<id>http://www.reddit.com/user/lemon_zest</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>i5fxh</id>		<published>2011-06-21T19:06:20Z</published>		<updated>2011-06-21T19:06:20Z</updated>		<title>_Laurana posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/trees/comments/i5fxh/chronic_cheesecake_recipe/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </gnip:rule>		</source>		<service:
 provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>i5fxh</id>			<title>Chronic Cheesecake Recipe...</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Crust&amp;lt;/strong&amp;gt;3 cups graham cracker crumbs,2 tablespoons sugar,3/4 cup cannabutter warmed to liquid.Press into the bottom of a large springform pan and set aside. I used a 10 inch pan. &amp;lt;/p&amp;gt;&amp;lt;p&amp;gt;&amp;lt;strong&amp;gt;Filling&amp;lt;/strong&amp;gt;2 lb. cream cheese (softened),1/2 cup of cannabutter warmed to liquid,1 lb. sour cream,1/2 cup whole milk,6 eggs (at room temperature),2 cups sugar,1 cup flour,pinch of salt,2 tablespoons lemon zest,2 tablespoons vanilla extract.&amp;lt;/p&amp;gt;&amp;lt;p&amp
 ;gt;&amp;lt;strong&amp;gt;Directions&amp;lt;/strong&amp;gt;This recipe takes a lot of mixing to completely mix ingredients ans ensure a smooth texture. Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole milk and cannabutter, mix for another 30 seconds. Add sugar, flour, salt, vanilla and lemon zest. Mix until smooth, scraping the sides of the bowl with a spatula occasionally. Then add eggs one at a time, mixing 30 seconds between each egg. Pour mixture into springform pan with crust. Now to keep the top of your cake from cracking, you can place the springform pan into a baking pan that has about 1 to 1 1/2 inches of water in it. I prefer using a really wet towel and wrapping it around the springform pan. Make sure to place it on a large cookie sheet. Bake @ 350f for 1 1/2 hours or until a wooden toothpick inserted into the center comes out clean. When you remove cake from oven, make sure to let it cool for about 1 hour before placing it in the fridge to
  chill. I would hill it for at least 3 hours before serving. Make sure to run a knife along the edge of the springform pan before opening it! Hope everyone enjoys. I use super potent cannabutter. It ensures my cake always turns out the best! Hope all you enjoy!&amp;lt;/p&amp;gt;&amp;lt;ul&amp;gt;&amp;lt;li&amp;gt;I had lots of request for this recipe. I hope everyone gets it that wanted it. I posted the recipe in the comment thread of my yesterdays post as well as in a new post. Ents Enjoy! &amp;lt;a href="http://www.reddit.com/r/trees/comments/i4pqp/i_made_a_chronic_cheesecake_10/"&amp;gt;Chronic Cheesecake&amp;lt;/a&amp;gt;&amp;lt;/li&amp;gt;&amp;lt;/ul&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/trees/comments/i5fxh/chronic_cheesecake_recipe/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics upVotes="88" downVotes="10"/>		</activity:object>		<author>			<name>_Lau
 rana</name>			<uri>http://www.reddit.com/user/_Laurana</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/_Laurana"/>			<id>http://www.reddit.com/user/_Laurana</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>y8nml</id>		<published>2012-08-15T03:37:25Z</published>		<updated>2012-08-15T03:37:25Z</updated>		<title>dinkleberg31 posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/todayilearned/comments/y8nml/til_that_the_average_lemon_has_2_oz_of_lemon/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </g
 nip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>y8nml</id>			<title>TIL that the average lemon has 2 oz of lemon juice and one ounce of zest.</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/todayilearned/comments/y8nml/til_that_the_average_lemon_has_2_oz_of_lemon/"/>			<link rel="enclosure" type="text/html" href="http://www.howmuchisin.com/produce_converters/how-much-juice-in-a-lemon"/>			<link rel="preview" type="image/jpeg" href="http://c.thumbs.redditmedia.com/HVK6XcTIOu3JIXD2.jpg"/>			<gnip:statistics upVotes="39" downVotes="15"/>		</activity:object>		<author>			<name>dinkleberg31</name>			<uri>http://www.reddit.com/user/dinkleberg31</uri>		</author>		<activity:actor>			<activity:object-type>http://act
 ivitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/dinkleberg31"/>			<id>http://www.reddit.com/user/dinkleberg31</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1ax1ok</id>		<published>2013-03-24T17:07:41Z</published>		<updated>2013-03-24T17:07:41Z</updated>		<title>NinjaNerd posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/wicked_edge/comments/1ax1ok/anyone_ever_try_lathering_up_a_soap_not/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </gnip:rule
 >		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1ax1ok</id>			<title>Anyone ever try lathering up a soap not specifically made for shaving?</title>			<content type="text/html">&amp;lt;!-- SC_OFF --&amp;gt;&amp;lt;div class="md"&amp;gt;&amp;lt;p&amp;gt;Anyone ever try this?  Curious to know whether you&amp;amp;#39;d be able to shave with any &amp;amp;#39;ol bar of soap, like one you&amp;amp;#39;d use in the shower for example (irish spring/zest/lever 2000).&amp;lt;/p&amp;gt;&amp;lt;/div&amp;gt;&amp;lt;!-- SC_ON --&amp;gt;</content>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/wicked_edge/comments/1ax1ok/anyone_ever_try_lathering_up_a_soap_not/"/>			<link rel="preview" type="image/jpeg" href="self"/>			<gnip:statistics u
 pVotes="7" downVotes="2"/>		</activity:object>		<author>			<name>NinjaNerd</name>			<uri>http://www.reddit.com/user/NinjaNerd</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/NinjaNerd"/>			<id>http://www.reddit.com/user/NinjaNerd</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1c8oyd</id>		<published>2013-04-13T01:04:28Z</published>		<updated>2013-04-13T01:04:28Z</updated>		<title>substance_d posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/1c8oyd/potato_and_yam_fries_with_parsley_and_lemon_zest/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </g
 nip:rule>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1c8oyd</id>			<title>Potato and yam fries, with parsley and lemon zest. [612X612]</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/1c8oyd/potato_and_yam_fries_with_parsley_and_lemon_zest/"/>			<link rel="enclosure" type="text/html" href="http://i.imgur.com/t90QUxO.jpg"/>			<link rel="preview" type="image/jpeg" href="http://b.thumbs.redditmedia.com/ejqTTzQzftksFL-g.jpg"/>			<gnip:statistics upVotes="38" downVotes="4"/>		</activity:object>		<author>			<name>substance_d</name>			<uri>http://www.reddit.com/user/substance_d</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>		
 	<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/substance_d"/>			<id>http://www.reddit.com/user/substance_d</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"  ""  --""</gnip:matching_rule>		</gnip:matching_rules>	</entry>
-<entry xmlns="http://www.w3.org/2005/Atom" xmlns:activity="http://activitystrea.ms/spec/1.0/"		xmlns:service="http://activitystrea.ms/service-provider" xmlns:thr="http://purl.org/syndication/thread/1.0"		xmlns:gnip="http://www.gnip.com/schemas/2010">		<id>1k9npy</id>		<published>2013-08-13T10:00:28Z</published>		<updated>2013-08-13T10:00:28Z</updated>		<title>YamiSC2 posted an article Reddit</title>		<category term="ArticlePosted" label="Article Posted"/>		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1k9npy/2013_wcg_korea_national_final_parting_vs_rain/"/>		<source>			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>			<updated>2013-08-28T17-54-04Z</updated>			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""            </gnip:ru
 le>		</source>		<service:provider>			<name>Reddit</name>			<uri>www.reddit.com</uri>		</service:provider>		<activity:verb>http://activitystrea.ms/schema/1.0/post</activity:verb>		<activity:object>			<activity:object-type>http://activitystrea.ms/schema/1.0/article</activity:object-type>			<id>1k9npy</id>			<title>2013 WCG Korea National Final :: PartinG vs. Rain :: Zest vs. INnoVation :: 02:00 (PST)</title>			<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1k9npy/2013_wcg_korea_national_final_parting_vs_rain/"/>			<link rel="enclosure" type="text/html" href="http://www.twitch.tv/ongamenet"/>			<link rel="preview" type="image/jpeg" href="http://a.thumbs.redditmedia.com/hP-mFydtcg2jEO3Z.jpg"/>			<gnip:statistics upVotes="40" downVotes="9"/>		</activity:object>		<author>			<name>YamiSC2</name>			<uri>http://www.reddit.com/user/YamiSC2</uri>		</author>		<activity:actor>			<activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:obj
 ect-type>			<link rel="alternate" type="text/html" length="0" href="http://www.reddit.com/user/YamiSC2"/>			<id>http://www.reddit.com/user/YamiSC2</id>		</activity:actor>		<gnip:matching_rules>			<gnip:matching_rule rel="source" tag="Zest">"zest"

<TRUNCATED>

[03/15] added support for FB user stream and user profile collection

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/49d00d7c/streams-contrib/streams-provider-facebook/src/test/resources/Facebook.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/resources/Facebook.json b/streams-contrib/streams-provider-facebook/src/test/resources/Facebook.json
new file mode 100644
index 0000000..26c86cb
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/test/resources/Facebook.json
@@ -0,0 +1,250 @@
+{"note":{"from":{"name":"Lauren Kay Bruce","id":"1795508919"},"application":{"name":"Facebook for iPhone","namespace":"fbiphone","id":"6628568379"},"id":"1795508919_4602522719840","created_time":"2013-08-27T23:36:39+0000","type":"status","updated_time":"2013-08-28T00:40:42+0000","privacy":{"value":""},"message":"Can Axe please make a deodorant for her, now."}},
+{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Emmanuel Fernandez Garnica","id":"100000114951549"},"id":"603849322999439_5966940","created_time":"2013-08-28T00:48:32+0000","message":"Que linea para delgada jaja"}},
+{"note":{"from":{"name":"Tevri Bagus P","id":"100002109812180"},"application":{"name":"Facebook for Every Phone","id":"139682082719810"},"id":"100002109812180_519296974817292","created_time":"2013-08-28T00:29:32+0000","type":"status","updated_time":"2013-08-28T00:29:32+0000","privacy":{"value":""},"message":"Yang ax btuh cnta mu seperti cnta ax yg besar buat kamu,\nyang ax syang mu seperti rsa syang ax buat kamu yg tulus,,\nax ga btuh yg lain selain itu doank,,"}},
+{"note":{"comments":{"data":[{"like_count":7,"user_likes":false,"can_remove":false,"from":{"name":"Patrik PipoPupy Tre?o","id":"100000141145825"},"id":"578610008862349_92416511","created_time":"2013-08-28T00:07:12+0000","message":"Uz ma zacinate jebat ...."},{"like_count":1,"user_likes":false,"can_remove":false,"from":{"name":"Duc Anh Le","id":"100004707488552"},"id":"578610008862349_92416523","created_time":"2013-08-28T00:11:33+0000","message":"J� bych toho Balea u? pustil k vod? ..."},{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Luko Eles","id":"100004483267342"},"id":"578610008862349_92416559","created_time":"2013-08-28T00:26:02+0000","message":"T�to spr�va je 2 dni star� :D :D"},{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"David Doktor","id":"100000140869004"},"id":"578610008862349_92416547","created_time":"2013-08-28T00:21:38+0000","message":"Kde berete tyhle zpr�vy?"},{"like_count":0,"user_likes":false,"can_remove":fa
 lse,"from":{"name":"Ondra's Berryss�n","id":"100004401523860"},"id":"578610008862349_92416556","created_time":"2013-08-28T00:25:05+0000","message":"Kone?n? snad!! :)"}],"paging":{"cursors":{"after":"MQ==","before":"NQ=="}}},"from":{"name":"Real-madrid.cz (Official)","category":"Website","id":"165574650165889"},"id":"165574650165889_578610008862349","created_time":"2013-08-28T00:05:51+0000","type":"status","updated_time":"2013-08-28T00:28:55+0000","privacy":{"value":""},"message":"Nejnov?j?� zpr�vy (Sky Sports Italia): Bale p?ijde za 87 milion? eur, Ronaldo z?stane nejdra??� v historii. Uvid�me, uvid�me. Bylo u? t� s�gy dost.","likes":{"data":[{"name":"Luciferus Milanovi?","id":"1423483222"},{"name":"Miroslav ?mok","id":"100000821972410"},{"name":"Denis Pichler","id":"100000222507112"},{"name":"Radek Han�k","id":"1231293572"},{"name":"Lukas Filcik","id":"100003190828171"},{"name":"Radek Tr??a Truhl�?","id":"100000263879140"},{"name":"F�fa Michi","id":"1000019429
 13726"},{"name":"Radim Macoszek","id":"100000411026577"},{"name":"Jan Han�sek","id":"100000467022164"},{"name":"Luk�?ek Pa?il","id":"1304584616"},{"name":"Maros Hric","id":"1794470539"},{"name":"Stanley Uli?ka-Ronaldo Hrica","id":"1180391866"},{"name":"Tom�? Svoboda","id":"1652694973"},{"name":"Dusan Gbelec","id":"100001729069935"},{"name":"Veronica Hrozov�","id":"100000021889547"},{"name":"Jakub Gelnar","id":"100000634354601"},{"name":"Milan Luk�?","id":"100000894167126"},{"name":"Hanina Vr�nov�","id":"100003307243536"},{"name":"Tom�? M�?o Medve?","id":"100000007192762"}],"paging":{"cursors":{"after":"MTAwMDAwMDA3MTkyNzYy","before":"MTQyMzQ4MzIyMg=="}}}}},
+{"comment":{"like_count":7,"user_likes":false,"can_remove":false,"from":{"name":"Patrik PipoPupy Tre?o","id":"100000141145825"},"id":"578610008862349_92416511","created_time":"2013-08-28T00:07:12+0000","message":"Uz ma zacinate jebat ...."}},
+{"comment":{"like_count":1,"user_likes":false,"can_remove":false,"from":{"name":"Duc Anh Le","id":"100004707488552"},"id":"578610008862349_92416523","created_time":"2013-08-28T00:11:33+0000","message":"J� bych toho Balea u? pustil k vod? ..."}},
+{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Luko Eles","id":"100004483267342"},"id":"578610008862349_92416559","created_time":"2013-08-28T00:26:02+0000","message":"T�to spr�va je 2 dni star� :D :D"}},
+{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"David Doktor","id":"100000140869004"},"id":"578610008862349_92416547","created_time":"2013-08-28T00:21:38+0000","message":"Kde berete tyhle zpr�vy?"}},
+{"comment":{"like_count":0,"user_likes":false,"can_remove":false,"from":{"name":"Ondra's Berryss�n","id":"100004401523860"},"id":"578610008862349_92416556","created_time":"2013-08-28T00:25:05+0000","message":"Kone?n? snad!! :)"}},
+{"photo":{"name":"Timeline Photos","story":"Marco Aur�lio Aur�lio shared FanatiCorinthians's photo.","from":{"name":"Marco Aur�lio Aur�lio","id":"100000629625325"},"application":{"name":"Links","id":"2309869772"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"0":[{"name":"Marco Aur�lio Aur�lio","id":"100000629625325","type":"user","length":21,"offset":0}],"29":[{"name":"FanatiCorinthians","id":"178543808920097","type":"page","length":17,"offset":29}]},"id":"100000629625325_628414700522845","created_time":"2013-08-28T00:47:52+0000","type":"photo","caption":"Recomendo: V�rus Corinthiano\r\nRecomendo: � o Time Do Povo, � o Coring�o\r\nRecomendo: CorinthiAMO","updated_time":"2013-08-28T00:47:52+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=468742883233520&set=a.178546862253125.39468.178543808920097&type=1","object_id":"468742883233520","message":"bic de ponta grossa , kkkkkkkkkkk","properties":[{"href
 ":"http://www.facebook.com/FanatiCorinthiansOficial?ref=stream","name":"By","text":"FanatiCorinthians"}],"picture":"http://photos-f.ak.fbcdn.net/hphotos-ak-ash3/1236687_468742883233520_1910998772_s.jpg"}},
+{"note":{"from":{"name":"Wellington Flores","id":"100004401480520"},"application":{"name":"Facebook for Android","namespace":"fbandroid","id":"350685531728"},"id":"100004401480520_222258177930897","created_time":"2013-08-28T00:47:39+0000","type":"status","updated_time":"2013-08-28T00:47:39+0000","privacy":{"value":""},"message":"Em Pensaar ..!!     Quee por apenas um diaa .  !    Iria ser assim  pra senpre.. .. !  \nSomoss convidados...!!  Aa entrar eem uma viidaa .. !!  Ee ninguem entraa  de bic�o ...!!"}},
+{"video":{"name":"Best inspirational love quotes","from":{"name":"Sajjad Khan","id":"100000877841588"},"application":{"name":"Share_bookmarklet","id":"5085647995"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yj/r/v2OnaTyTQZE.gif","id":"100000877841588_590446897661233","created_time":"2013-08-28T00:47:24+0000","type":"video","caption":"www.youtube.com","updated_time":"2013-08-28T00:47:24+0000","privacy":{"value":""},"description":"Best quotes and saying about love embed in beautiful roses photos, inspiration love quotes and pictures, best quotes about love, cute love quotes. Visit: htt...","link":"http://www.youtube.com/watch?v=e3VPZepU6V0&sns=fb","source":"http://www.youtube.com/v/e3VPZepU6V0?version=3&autohide=1&autoplay=1","message":"its for you bic\r\n","picture":"http://external.ak.fbcdn.net/safe_image.php?d=AQDFvLV-yo-lT-CO&w=130&h=130&url=http%3A%2F%2Fi1.ytimg.com%2Fvi%2Fe3VPZepU6V0%2Fhqdefault.jpg%3Ffeature%3Dog"}},
+{"note":{"from":{"name":"Um Dia de F�ria","category":"Community","id":"201691339996455"},"id":"201691339996455_210984119067177","created_time":"2013-08-28T00:46:35+0000","type":"status","updated_time":"2013-08-28T00:46:35+0000","privacy":{"value":""},"message":"Um dia de F�ria 17 - A Magrela da Oric� e a Bala Perdida\n-------------------------------------------------------------------\n\nEra o s�bado do desfile das campe�s do Carnaval de 1994, ainda tinha uma ressaca do Carnaval para curtir no Largo do Bic�o. Meu amigo Junior MCGyver havia me chamado para ir a Br�s de Pina para ficar com umas meninas e eu fui, claro. Mas chegando l� s� tinha uma e ele n�o quis compartilhar, com a promessa de que o que era meu estava guardado. Mais de uma hora depois e j� cansado de espera-lo a dar uns pegas na madame, voltamos para a Vila da Penha, onde nos encontramos com os demais delinquentes da �poca: Gustavo, Alisson, Luciano, Galdino e por a� vai.\nAo chegarmos na fes
 ta, ele me apresentou a famosa Magrela da Oric� que eu nem preciso citar o nome. Quando eu me deparei com situa��o, confesso que me senti, no m�nimo, surpreso, pois nunca havia conhecido na vida uma menina t�o magra quanto aquela. Ela j� tratou de se apresentar e passou a me chamar de fofo e mostrou total interesse em mim e passou a me perseguir durante toda a festa. Estava me sentindo a verdadeira v�tima de um psicopata Serial Killer. Quanto mais eu despistava, mais ela me encontrava. O grande problema � que na adolesc�ncia, quanto mais o tempo passa mais voc� bebe e quanto mais voc� bebe mais gostosa voc� come�a a achar as mulheres, qualquer uma delas. \nN�o sei hoje, mas nos anos 90 era muito comum ter carros cheio de batidas no caput, e voc� pagava uma miserinha por uma garrafa e enchia a cara a noite inteira, sendo que voc� nunca bebia sozinho, pois os amigos sempre aparecia com um copinho descart�vel para ser arregrado.\nE ao bater das doze ba
 daladas noturnas, a carruagem virou ab�bora e as minhas resist�ncias viraram p�. A Magrela pegou na minha m�o e disse: � agora!!!! Me aproximei e mais uma vez consegui resistir aos seus poderes hipn�ticos,,, s� n�o sabia at� quando. Passava desfilando com ela e todos me homenageavam pela coragem e bravura e ela come�ou a ficar impaciente e disse que se eu n�o queria, bastava falar,,,, mas fui um fraco e ao mesmo tempo um cavalheiro, pois n�o poderia deixar a menina magoada. Ent�o bolei um plano mirabolante. O Alan e a sua irm� estavam indo embora, ent�o falei pra ela que far�amos companhia para os dois e depois selar�amos o esperado beijo da noite. Era o local perfeito pra n�o ser flagrado por toda sociedade Viladapenhana ou ent�o o momento mais prop�cio para sair correndo.\nFoi quando rolou um tumulto violento e uma correria danada,,, todos desembestados e eu sem saber o que estava acontecendo,,, ouvi tiros, gritos e a sensa��o mais louca
  que eu j� senti na minha vida, tentei correr mais levei uma porrada na nuca, ouvi som de trinca-ferro e os olhos come�aram a triplicar as imagens. O som ficou mais lento, como se reduzisse a rota��o do disco de vinil. Olhava para os lados e via pessoas ainda correndo. Tentei dar o pr�ximo passo, mas n�o tinha mais for�a nas pernas. Senti o ch�o chegando muito vagarosamente, coloquei meus bra�os para amortecer a queda e ao tocar o asfalto, como num passe de m�gica, tudo voltou ao normal. Eu ainda tentei rastejar para fugir do tiroteio, mas um transeunte me segurou e me informou aquilo que eu n�o queria pensar: Eu havia levado um tiro e sangrava muito. Fui virado pra cima e vi as pessoas se amontoarem em cima de mim. Rasgaram minha blusa para arejar, tiraram meus t�nis para arejar, e por sinal at� hoje eu n�o os vi de volta, um Nike novinho, do Alisson... \nO boato dos baleados se espalhou e os carniceiros dos meus amigos vieram correndo pra ver quem era o
  bandido alvejado e se deparam comigo deitado sobre um mar de sangue. Todos se apavoraram e o Alisson, no seu melhor estilo capoeira, furou o cord�o de isolamento e veio ficar comigo. Eu estava com um medo fodido de morrer, olhava pro lado e via o sangue escorrer pela rua abaixo. A �nica viatura de pol�cia que havia no local,  levou um baleado na garganta e um gr�vida alvejada de rasp�o na barriga. E eu fiquei l�, esperando a minha hora chegar. Foi quando um santo taxista se ofereceu pra me levar ao hospital. Ao tentar levantar, eu nunca poderia imaginar que a minha cabe�a pesava tanto, acho que a bala rompeu alguns ligamentos de sustenta��o dos m�sculos do pesco�o. \nEntrei no carro com muito sacrif�cio e sentia vertigens incontrol�veis, iria desfalecer ou mesmo falecer a qualquer momento, o problema � que a luz branca n�o veio, ent�o pensei: Fudeu, vou pro inferno. O Alisson tentava me animar, me fazia cantar Legi�o Urbana: Que Legi�o Urbana �
  o caralho,,, estou morrendo cara, voc� n�o est� vendo???.... E os olhos temiam em fechar. O hospital Get�lio Vargas nunca foi t�o longe,,, levamos uma eternidade para chegar,,,, essa era a sensa��o. Ao chegar no hospital, n�o tinha leito, me deitaram numa maca met�lica fria, sem camisa e com hipotermia, por causa da perda de sangue. Fui operado ali mesmo, extra�ram a bala, embalaram numa gaze e me deram. Me senti uma mam�e no parto ao ter seu filho nos bra�os. Olhei pra bala e disse: Filha da Puta,,, quase me matou.\nE pra fechar a noite com chave de ouro, o Alisson me pergunta: \n- E a magra, pegou?\n- N�o, escapei,,,, ufa... santa bala perdida!!!!"}},
+{"note":{"from":{"name":"Dakoda Mandujano","id":"100002494114668"},"application":{"name":"Mobile","id":"2915120374"},"id":"100002494114668_501954526564347","created_time":"2013-08-28T00:45:48+0000","type":"status","updated_time":"2013-08-28T00:47:43+0000","privacy":{"value":""},"message":"i need my Bic sucked right now lol"}},
+{"note":{"from":{"name":"Gi?t M?c M�a H�","id":"100006547540821"},"id":"100006547540821_1389297757965129","created_time":"2013-08-28T00:41:31+0000","type":"status","updated_time":"2013-08-28T00:41:31+0000","privacy":{"value":""},"message":"Ch?c ch?n h?n anh to�n m?c t�m \nB� kh�ng tin ch?m th? v�i l?n \nM?t l� ???c th?y cho m??i ?i?m \nHai l� tim b� b? ... b�ng khu�ng\nAnh l�m th? to�n b?ng vi?t bic \nL�m quen ch?a bi?t vi?t b?ng g� \nS?c nh? ng�y x?a c�n l? m?c \nS�n tr??ng t�m ??m m?t c�y si\n\nB�i h�t n�o \"s?i th??ng s?i nh?\" \nC�n th? anh \"gi?t nh? gi?t th??ng\" \nM?t h�m anh bi?n th�nh quy?n v? \nT�nh nguy?n y�u gi?t m?c t�m ?ang bu?n"}},
+{"note":{"from":{"name":"Trang T�mat?","id":"100002216264738"},"application":{"name":"Facebook for iPhone","namespace":"fbiphone","id":"6628568379"},"id":"100002216264738_499056076844930","created_time":"2013-08-28T00:40:48+0000","type":"status","updated_time":"2013-08-28T00:40:48+0000","privacy":{"value":""},"message":"Tinh m? s??ng ?i t?p e r� bic\n...c�i c?m gi�c t?p xong ng ??t h?t m? h�i r�i lao xe vun v�t => m�t r??i iii\n=> th?t l� th�ch >:'D<"}},
+{"note":{"from":{"name":"Richly Awthentic","id":"100001409130885"},"id":"100001409130885_589039464486353","created_time":"2013-08-28T00:34:26+0000","type":"status","updated_time":"2013-08-28T00:34:26+0000","privacy":{"value":""},"message":"fire in the HOLE!..*stirkes BIC*"}},
+{"note":{"from":{"name":"Black Friday Cyber Monday on Sale","category":"Product/service","id":"1376926415869828"},"id":"1376926415869828_1382427048653098","created_time":"2013-08-28T00:34:10+0000","type":"status","updated_time":"2013-08-28T00:34:10+0000","privacy":{"value":""},"message":"BIC Sport Adjustable Stand Up Paddle Blade\n\nCHECK TO FIND LOW PRICE >> http://tinyurl.com/gbkj4qsx/B009VYHJRI\n\nBIC Sport Adjustable Stand Up Paddle Blade The four Most Well-known Jugglers At any time - Find out From Them, YOU Could Be Subsequent! Have you at any time dreamed of remaining the future legend in juggling? I have. So I wrote this manual on the four most famed jugglers, and their insider secrets to turning out to be the legends they had been. Understand from all those insider secrets, and you can be the following legend in juggling! BIC Sport Adjustable Stand Up Paddle Blade.  The Journey Privilege of Motorized Paragliding Paragliding experts already have the liberty to fly like birds
  on their have at their own prices and travel as much as they can. They appreciate the privilege of touring or crossing nations. BIC Sport Adjustable Stand Up Paddle Blade.  Lacrosse Drills - Shooting Drills From Cornell Lacrosse These are great lacrosse drills for increasing the capturing skills of your lacrosse players. Lacrosse coaches can make their teams much more powerful at scoring additional plans with these easy drills. These drills came in an job interview with Coach Tambroni from Cornell Lacrosse."}},
+{"note":{"from":{"name":"Cyber Monday Black Friday Online","category":"Product/service","id":"565321923532890"},"id":"565321923532890_568015023263580","created_time":"2013-08-28T00:33:37+0000","type":"status","updated_time":"2013-08-28T00:33:37+0000","privacy":{"value":""},"message":"BIC Beach Kayak Paddle (1 Piece)\n\nCONTINUE TO SEE LOW PRICE >> http://tinyurl.com/rerqq91g/B004XFVOWG\n\nBIC Beach Kayak Paddle (1 Piece) Yukon one thousand - The Longest Canoe and Kayak Race The Yukon a thousand is the longest canoe and kayak race in the earth. Racers contend with seven to eight days of steady paddling with only a few hrs of rest each individual night time. BIC Beach Kayak Paddle (1 Piece).  Athletics - Bringing Americans With each other Sports have touched and have achieved out to each and every side of American existence. Due to the greater availability of medium made use of to observe and share these activities, (Tv set and the Internet), sporting activities encounters and reminis
 cences have been shared a lot more and with hundreds of thousands of Americans in 1 variety or another. We have found dramatic wins, losses, championships, terrific plays, and not so terrific performs. People adore sports activities and the athletics that perform them and really like to share and communicate about these good moments. BIC Beach Kayak Paddle (1 Piece).  The Decline of Cannabis Details Hashish has been the most morally grey topic place for a long time and the information and facts you acquire on the subject can be really far from the reality. Certainly in most nations spherical the planet Hashish is Unlawful, however individuals are even now working with it so there really should be a bank of details about it."}},
+{"note":{"from":{"name":"Gabriel Paul Pierce-Lackey","id":"100003952643883"},"id":"100003952643883_293782680763484","created_time":"2013-08-28T00:30:39+0000","type":"status","updated_time":"2013-08-28T00:38:29+0000","privacy":{"value":""},"message":"If you snag lighters, I hate you"}},
+{"note":{"from":{"name":"Udin Zeoss","id":"100004916228419"},"application":{"name":"Mobile","id":"2915120374"},"id":"100004916228419_193897570784116","created_time":"2013-08-28T00:30:09+0000","type":"status","updated_time":"2013-08-28T00:30:09+0000","privacy":{"value":""},"place":{"name":"PaBriK paNg UjuNg Na ?? BiC-PurwakarTa","location":{"city":"Cikampek","latitude":-6.440615,"zip":"","country":"Indonesia","street":"","longitude":107.4492325},"id":"469609653049663"},"message":"Santai dl sblm gwe"}},
+{"note":{"from":{"name":"Andrel Barbour Doing Her","id":"100000828229524"},"application":{"name":"Mobile","id":"2915120374"},"id":"100000828229524_567828116588124","created_time":"2013-08-28T00:29:53+0000","type":"status","updated_time":"2013-08-28T00:48:36+0000","privacy":{"value":""},"message":"ALL GLORY....I DNT CARE...OR GIVE A FLYIN FLOCK WHAT NOBODY ELSE DO OR SAY AS LONG MY GLORY NAME ANIT IN IT....1 BIRD DNT KILL A STONE...AND I THAT BIC BIRD AND GOING OR WORRYIN ABOUT NOTHING N MY LIL BRO..EVERETTE..BKA...DANK VOICE...SAY IT AGAIN...I ANIT WORRYIN ABOUT NOTHING....OW OW OW OW OW OW...JUST BRING THE SMOKES I GOT THE WRAP...BRING THE ALCOHOL...I GOT THE CUPS...BRING THE BARBQUE I GOT THE PLATES...FOLKS...SPOONS AND NAPKINS...CANT GET NO MATTER THEN THAT....LMAO..."}},
+{"note":{"from":{"name":"Blackie Ibrahim","id":"100001611878761"},"application":{"name":"Mobile","id":"2915120374"},"id":"100001611878761_569548039775587","created_time":"2013-08-28T00:29:26+0000","type":"status","updated_time":"2013-08-28T00:29:26+0000","privacy":{"value":""},"message":"THIS ARE DAYZ 2  BE REMEMBRED!!!!\n\n  I miss d days when we went to\nschool, lined up & D headmistress &\nteachers inspect our nails & uniform\n& den we match to our classrooms.\n? D days of Natco Biscuit, Iced\nColored water tied in nylon we called\nit \"kulaid\"\n? D days of pehpeh mint\n? D days of WAN POT,correction corner(paddy nor piss dae!)\n? Hay, i remember those days when\none block coin na money,when\ngroundnut was 50 leone.&\nsweet was 2,3,4 for 50 Leone\n? days of messing game,forl shit(police,find me d thief), police and thief\n? D days of mummy & daddy play\n? days when we use to build houses\nwith sand, play borlorgie game, \nskipping, street ball stop! U don remember!\n?those days 
 wen we used to fly kite\non streets, wen boys used to use d\nMilk cups to make cars & their\ndaddy's hanger as d steering\n?those days when rubber band(fark) was\nstock exchange\n?days when mortal kombat,street fighter,snake and the monkey shadow,power rangers, spider man were our\nfavourite movies\n? D days of Pepsi soft drinks & milo and maltina advert on black and white tv\n? when we say ''ar stiff,nor stiff o. Wan cup wata! '' and slapping our colleagues saying 'fresh leg o'\n? Those days when bic pen\nwas d best\nD days wen we used to drink water\nfrom d tap even suck out d water if its\nnot coming out\n? D days when SLBS will show\nrainbow color for 30 mins then\nnational anthem before they resume\nprogram @ 4pm\n? D days wen silectto were d best footwares\n?Days wen we all sing ''neneh binta dae sell chi gum, chocolate,die mint,sweet!\nI'm really\nproud to have experienced\nall this. If u r nt smiling it means u\nwere nt born in my generation.\nAdd yours and pass the fun arou
 nd, hey heee haahaaaaaa opopinaaaazzzzz"}},
+{"note":{"from":{"name":"William George","id":"100003820913189"},"application":{"name":"Facebook for Android","namespace":"fbandroid","id":"350685531728"},"id":"100003820913189_320249768112384","created_time":"2013-08-28T00:29:14+0000","type":"status","updated_time":"2013-08-28T00:29:14+0000","privacy":{"value":""},"message":"Felling bad today have no meds for my diabetes because I am broke I need some help I have some u of l tee shirts and hats u of k all so all brand new will sell cheap all so bic  lighters if anyone can help me out hmu on fb any help would be great I feel like I am desperate so ty"}},
+{"photo":{"from":{"name":"Thami de Oliveira","id":"100003470164026"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yz/r/StEh3RhPvjk.gif","id":"100003470164026_390972087695127","created_time":"2013-08-28T00:29:07+0000","type":"photo","updated_time":"2013-08-28T00:44:13+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=390971777695158&set=pcb.390972087695127&type=1&relevant_count=2","object_id":"390971777695158","message":"MENINAS QUE TIVEREM INTERESSE NESSE VESTIDO TEM NA COR BRANCO,PRETO,AMARELO,AZUL BIC,ROSA PINK E AZUL PISCINA , COMENTA COM A COR QUE FOR QUERER PRA EU PODER FAZER A ENCOMENDA, S� VOU TRAZER DE QUEM COMENTAR !!!","picture":"http://photos-a.xx.fbcdn.net/hphotos-ash3/1234956_390971777695158_1843537076_t.jpg"}},
+{"note":{"from":{"name":"Black Friday Cyber Monday on Sale","category":"Product/service","id":"1376926415869828"},"id":"1376926415869828_1382424758653327","created_time":"2013-08-28T00:28:52+0000","type":"status","updated_time":"2013-08-28T00:28:52+0000","privacy":{"value":""},"message":"BIC Sport ACE-TEC Stand-Up Paddleboard\n\nCONTINUE FOR SPECIAL PRICE >> http://tinyurl.com/gbkj4qsx/B005SH9TNO\n\nBIC Sport ACE-TEC Stand-Up Paddleboard Types of Geocaching Caches The satisfaction achieved when enjoying geocaching usually lies on the discovery of what a cache incorporates. This serves as the reward for all the terrain that a cacher has successfully conquered. Caches come in different varieties and usually include a logbook (for greater caches) or log paper (for more compact caches). BIC Sport ACE-TEC Stand-Up Paddleboard.  Leisure at Relatives Reunions Looking for a risk-free outside game that can be performed by folks of all ages and qualities? Then search no further than cornhole.
  Cornhole is a addictive activity that is easy to master and a ton of enjoyable to play. BIC Sport ACE-TEC Stand-Up Paddleboard.  How to Prevent Soaked Ft When Going for walks in the Uk Most of my hiking and strolling usually takes position in the British isles. Locations to stroll this kind of as the Lake District, Scotland and Snowdonia are my favourite spots. These locations are wonderful."}},
+{"note":{"from":{"name":"Fahri  Kuliev","id":"100003757786565"},"id":"100003757786565_336091023192821","created_time":"2013-08-28T00:27:43+0000","type":"status","updated_time":"2013-08-28T00:27:43+0000","privacy":{"value":""},"message":"F?XR? M�SL�M\nBANKS CHA?RMAN\nMini chronologial novel\nBirinci hiss?\n V  \n H?s?n Qas?movi�in yan?nda,  m?n, t?k deyildim. T?fti? ?�b?sinin r?isi d? var idi. H?l? o,  t?fti? ?�b?sinin r?isini dinl?yirdi. T?fti? ?�b?sinin r?isi tutatl? s�but v? d?lill?rl?, sanki �z dedikl?rini �z� d? t?sdiq edirdi. \n O, �z�n� H?s?n Qas?movi�? tutaraq, - bu yolda? birinci yoxlama i?ind? i?tirak edir, akt mateiallar?n? oxudum, birinci d?f? ���n yoxlama materiallar? pis deyil. Ancaq veril?n m?lumata ?sas?n, qeyd edim ki, bu yolda? yerl?rd? tamam ba?qa i?l?rl? m???ul olur. Bu, h?m onun t?fti?�i ad?na, h?m d? i?l?diyi t??kilat?n ad?na l?k? g?tir? bil?r. O, yoxlama d�vr�nd?, ?eir yazmaqla m???ul olub, yerli radio verli?l?rind? g�n
 a??r? �?x??lar edib, rayon q?zetind? ?eirl?r d?rc etdirib. Fikirl??in ki, bu i?l?r bel? davam ed?rs?, onda t?fti? i?i hans? m�sib?tl?rl? �zl??? bil?r, h?m keyfiyy?t c?h?td?n, h?m d? yoxlan?lan t??kilata  qar??  t?l?bd?n v? s. s�hb?t gedir. Ax?, bu bizim i?imizl? daban-dabana zidd bir m?s?l?dir. Sonra yerl?rd?n biz? eham ed?c?kl?r ki, siz bura t?fti?�i g�nd?rmisiniz, yoxsa, jurnalist, yaz?�?, ?air.\n M?n bunlar?n hardan qaynaqland???n? bilirdim v? �z-�z�m? fikirl??irdim ki, i?? girdiyim g�n�n s?h?ri t?fti? ?�b?sinin r?isi,  t??kilat?n, b�t�n rayonlara ged?c?k  m?ktublar?n?, m?n d? daxil olmaqla bir ne�? n?f?rd?n xahi? etdi ki, ba? po�t idar?sin? apar?b t?hvil ver?k. Bu m?s?l?y? m?n etiraz ed?r?k, - t??kilat?n ma??n? yoxdu, n?di, -demi?dim  v? m�?yy?n bir m?saf?y?, �z� d? ??h?rin m?rk?zind?,  ??ll?nib y�k aparma?? �z�m? s????d?rmam??d?m. V? m?n �z pay?ma d�??n m?ktublar?  taksi tutaraq po�ta aparm??d?m, bu da ?�b?d?  m?nd?n gizli s
 �z-s�hb?t? s?b?b olmu?du. �st?lik d? b�t�n bunlar m?nim yek?xanal???m kimi d?y?rl?ndirilmi?di. O, dan??d?qca h?min ?hvalat yad?ma d�?d� v? �zl�y�md? fikirl??dim ki, r?isin ?lin? gir?v? d�?�b,  hay?f?n? bu yolla v? bird?n �?xmaq ist?yir. G�zl?rim onun �z�nd?, qulaqlar?m is? onun gileyind? olmas?na baxmayaraq, sakit oturmu?dum, ��nki h?r ?ey H?s?n Qas?movi�in ver?c?yi q?rardan as?l? oldu?unu, art?q, g�t�r-qoy etmi?dim. Ancaq b�t�n bunlara baxmayaraq, i�imd? bir rahatl?q da var idi v? bu rahatl?q t?k m?nim i�imd?kil?rl? ba?l? deyildi, h?m d? t?fti? ?�b?sinin r?isinin H?s?n Qas?movi�l? olan m?nfi m�nasib?tl?ri il? ba?l? bir m?s?l? idi.\n H?s?n Qas?movi� d?  ona diqq?tl? qulaq asmas?na baxmayaraq, sanki onun ?ikay?t xarakterli gileyin? yox, ?a? atl? da?d?mirin na??l?na? qulaq as?rd?.  T?fti? r?isi art?q hiss etdi ki, dan??ma?a bir ?ey qalmad?, oturdu?u stulda yayxand? v? a??r bir y�k�n alt?ndan azad olmu? kimi, d?rind?n bir n?
 f?s ald?. O, sif?tind? bir ovuc qom olmu?, k�lg? r?ngind? t?b?ss�m� il?  adamda ikrah hissi do?ururdu. O, el? adam? bo?an bu ikrah hissi il? d? arxay?n-arxay?n H?s?n Qas?movi�in �z�n? bax?b qalm??d?. Qalm??d? ki, H?s?n Qas?movi� bu m?s?l?y? nec? yana?acaq. Bird?n-bir?, m?n?  el? g?ldi ki, bu adam canl? deyil, yapon d?zgahlar?nda istehsal olunmu? robotdu v? d�ym? il? avtomatla?d?r?lm?? formada  idar? olunur, laz?m olanda dan???r, laz?m olmayanda susur.\n H?s?n Qas?movi� is? yax?? i? g�rm?di, ona qulaq asandan sonra, onun getm?yin? i?ar? etdi. G�r�n�r o, m?ni r?isin yan?nda ya pis v?ziyy?td? qoymaq ist?mirdi, ya da  fikirl??di ki, m?n d? insanam, bu m?s?l?y? m?nim d? �z yana?mam ola bil?r v? onda t?fti? ?�b?sinin r?isi pis v?ziyy?td? qala bil?r. B?lk? d? o, m?niml? t?kb?t?k dan??maq da, m?ni g?l?c?k i?l?r ���n t?dqiq etm?k ist?yirdi, fikirl??dim. Bu o q?d?r d? �ox �?km?di, H?s?n Qas?movi� sa? �iyini �std? m?n? t?r?f d�nd�: - a ki?i, m?
 n g?lmi??m rayona? rabo�i? paltarda, s?n is? kostyumda otrmusan, bo?az?nda da qalstuk, o q?z da b�yr�nd?n bel? ke�ir, el? ke�ir, sa�lar? da s?nin �iynind? g?z?-g?z?.  S?n d? feyziyab olub, s�z qo?ursan, ?eir yaz?rsan, n? bilim daha n?l?rl? m???ul olursan. Birinci n�vb?d?, o q?z ba? m�hasibdi v? s?n d? onun s?n?dl?rini yoxlay?rsan, bu formada s?n hans? n�qsanlar? �z? �?xarda bil?rs?n. O q?z s?ni bax??lar?yla aldada-aldada ciddi n�qsanlar?, m?nims?m?l?ri d? s?nd?n yay?nd?ra bil?r. Bu, bu rayonda olmasa da, ba?qa bir rayondak? olacaqd?r. M?n s?ni i? g�rm?y? g�nd?rmi??m, s?n d? l?,..l?,..lo,..lo...i?l?rl? m???ulsan.  Bu, b?s el?mir kimi, q?zet bel? getdi, radio bel? getdi. G�rd�n ki, s?h?rd?n t?fti? ?�b?sinin r?isi durub-dayanmadan �y�d�b t�k�r, m?n burda s?n? g�z??t? ged?c?m, ondan i? t?l?b ed?mm?y?c?m, ona g�z??t? gets?m, s?nin ?m?k kitab�an korlanacaq.\n M?n, verilic?k cavablar?m?, i�imd? yava?-yava? g�t�r-qoy edirdim. V? bu
  cavablar? g�t�r-qoy etdiyim? g�r? d? bir q?d?r arxay?nla?m??d?m v? g�zl?yirdim ki, H?s?n Qas?movi� s�z�n� deyib qurtars?n, m?n d? �z d?lil-s�butlar?mla onu inand?r?m ki, - H?s?n Qas?movi�, ax? el? deyil, bu i? m?nim ?sas i?im? mane�ilik etmir, ?ksin?, bu i? d?, o i? d? bir-brini tamamlay?r.\n O, art?q, sar???n v? cod sif?tind? gizl?tdiyi g�y v? z?himli g�zl?rini m?n? zill?mi?di v? m?nim n? dey?c?yimi s?brsizlikl? g�zl?yirdi.\n M?n, ani olaraq �z�m� y????d?rd?m v? n?d?n ba?layaca??m? g�t�r-qoy etm?y? �al??d?m, ��nki qar??mda ya?l? bir insan?n oldu?unu d�?�nd�m. V? bir q?d?r ehtiyatla, inand?r?c? ??kild? onun �z�n? baxd?m, - H?s?n Qas?movi�, -dedim,-yaz?-pozu he� vaxt d?qiq sah?l?r? mane�ilik t�r?tmir, ?ksin?, bir-birin? b?lk? d? dayaq durur, ��nki burda m?n?viyyat �l��l?ri d?, h?yat �l��l?ri d? bir-birini tamamlay?r v? m�hk?m bir ba?lant? yarad?r. Ist?r d�nyada v? ist?rs? d? yax?n ??rq ?d?biyyat?nda 
 taa q?dimd?n yaz?-pozu adamlar? h?mi?? d?qiq v? ciddi i?l?rl? m???ul olmas?na baxmayaraq, ?eirl? d?, yaz?�?l?qla da, b?st?karl?qla da, r?ssaml?qla da, n?qqa?l?qla da v? s. m???ul olublar. B�y�k T�rk imperiyas?n?n ?ah? Sultan Suleyman h?m d? d�vr�n�n yax?? ?airl?rind?n olmu?du. Az?rbaycan imperiyas?n?n yarad?c?s? ?ah ?smay?l X?tai s?rk?rd? v? ?ah olmas?ndan as?l? olmayaraq �z ?airliyi il? Az?rbaycan ?d?biyyat?n?n apar?c?lar?ndan biriydi. Bununla yana??, m?n b�t�n bunlar? sadalamaq da �z�m? haqq qazand?rmaq ist?mir?m. M?s?l? bundad?r ki, m?n b�t�n bu yaz?-pozu m?s?l?si il? i?d?n sonra m???ul oluram. M?n, dan???b qurtard???m? hiss etm?dim, birc? onu hiss etdim ki, bu sar???n ki?inin �z�nd? n?s?, m?n? qar?? bir xo? ?hval-ruiyy?  ?m?l? g?lmi?di. V? m?n bunu, �zl�y�md? raz?l?q hissi kimi, �z�m�n d? q?l?b?m kimi hiss etdim. Bu da??n?q hissl?rimin alt?nda o, he� n? dem?d?n getm?yim? i?ar? etdi. �z�md?n raz? qalsam da i�imd? bir narahatl?q 
 hissi d? yox deyildi v? bu hissl?rl? d? otaqdan �?xd?m.  \n  Sanki �z�m? arxay?n idim, ancaq bu arxay?n�?l???n f?rqini saf-��r�k ed? bilmirdim. Birc? onu bildim ki, t?fti? ?�b?sind? stulumda oturmu?am. T?fti? ?�b?sinin r?isi d? t?fti?�il?rl? hans?sa, m�r?kk?b bir sual?n h?llini ayd?nla?d?r?r. M?n? �atan o oldu ki, s�hb?t, m�hasibat hesablar?ndan birinin hans? hesabla m�xabirl??m?sind?n gedir.  Bel? ba?a d�?d�m ki, hesab?n debetini d�zg�n t?yin  etmi?dil?r, ancaq onun hans? hesab?n kreditin? m�xabirl??m?si m�bahis?li qalm??d?. Donmu? bax??lardan hiss etdim ki, t?fti? ?�b?sinin r?isinin �z� d? bunu d�z-?m?lli bilmir. M?n, �zl�y�md? g�t�r-qoy etm?y? �al??d?m, ancaq m?n d? bu m?s?l?l?rin d?rkind?n �ox uzaqda idim. \n\n VI\n\n Art?q Xank?ndind? idik. �� n?f?r idik. M?n, kadrlar ?�b?sinin i?�isi Namik v? h�quq?�nas ?lqar. Namik t?bi?t?n it oynadan idi. Ilqar?n �z� kimi d? t?bi?ti var idi, haynan hay kimiydi, y
 aynan yay kimiydi, yay?n? �?kib oxunu gizl?dirdi. Kimliyini bilm?y?n bir adam idi. Gah ?sirdi, gah k�s�rd�, gah da as?b k?sirdi, h?mi?? d? adam? aldatma?a �al???rd?. O, m?niml? bir az xo? r?ftar ed?n kimi, h?r ?eyi unudurdum v? onunla da tez raz?la??rd?m, bu da m?n? baha ba?a g?lirdi. Sonra hiss edirdim ki, d�?m?ni hardasa axtarmaqla deyil, d�?m?n el? �z i�imizd?ymi?. Qan qrupu �ox k�p?ko?lu n?sn?ymi?.  Bu t�rkl�k d? bizim ba??m?za b?la olub. Boz qurdu �z�m�z? bayraq etmi?ik, ulam-ulam ulay?r?q, ulaya-ulaya da vaxt apar?r?q, bu vaxt ?rzind? d? hay ba??m?z? k?sib qoyur dizimizin �st�n?.  Arxada qal?b gecik?nl?rmiz is? salidorla yum?altd??? �?km?l?rini s�r�y?-s�r�y?  v? soyuqqanl?, soyuqqanl? yanda, y�r?d? can?n? gir?l?yir.\n Hay olmas?na baxmayaraq Razmik m?nim ���n yax?? adam t?siri ba???lay?rd?. Bu, b?lk? d? m?n? el? g?lirdi. ��nki o, m?nim ���n yax?? adam, haylar ���n pis adama ��vrilmi?di. Fikirl??irdim ki,
  b?lk? qan damar? bizd?ndi. B�t�n bunlar da ehtimal xarakteri da??y?rd?. O, m?n? yalman?b �z�n�nk�l?ri sat?rd?, bu is? m?nd? ba?qa bir t?sir yarad?rd?. Fikirl??dim ki, �z�n�nk�n� satan adam, sabah m?nim �z�m� d? pis v?ziyy?td? qoya bil?r v? ?�k�r edirdim ki, h?l?  onun yan?nda bir q?bah?tim yoxdur. Ikimiz bir otaqda oturmu?duq, onun �z�n? baxd?m, y?ni baxd?m ki, o, el? bel? i?�i deyil, h?r halda bizim ba? idar?mizin ?tat vahidi kimi, m?sul bir v?zif? da??y?r. V? ona g�r?, birinci d?f? oldu?um bu Xank?ndind?ki  t??kilat?n idar? sistemin? aid m?s?l?l?rl? yax?ndan tan?? olma??m ���n onun yard?m?na ehtiyac?m var idi.  O, el? bil m?nim i�imd?n ke�?nl?ri d?rk etmi?di v? ?lind?ki s?n?di stolun �st�n? qoyub, s�z? ba?lad?:\n O, - eli man siza bir ne�a pakt veracam, siz da eli ona uy?un i? gorun ha. Birincisi, siz eli soru?a bilars?n?z ki, ?u?a ?�bas?nda i?layan i?�ilar?n mukafat?n? niy? kas?blar ha. Eli siza qaranl?q qalmas?n deya
  izah edim ha. ?u?a rayonu hesabat? be? g�n gecikdirib ha, ona g�ra da m�kafatlar?n? kas?blar ha, ancaq Qadrut ?�b?si hesabat? on be? g�n gecikdirb ha, ancaq bunlar �z adamlar?d? deya tarixda duzali? ediblar v? hesabat vaxt?nda verilmi? kimi qeydiyyata al?n?b v? saxta yolla da onlara mukafat veriblar ha.\n M?n, bunlar? bilirdim ancaq d?rinl??dirm?k ���n onun �z�n? baxd?m:\n - bunlar? onlara nec? izah edim ki, s?nin informasiyan oldu?unu bilm?sinl?r, -dedim\n Razmik , - eli, siz ba? m�hasibi �a??r?n v? ondan soru?un ha, o da izah edacak ki, el?di, bel?di. Onda siz h?r iki ?�ban?n hesabat?n? tahlil edin ha, onda g�racaks?n?z k?, ?u?an?n gecikma tarixi oldu?u kimi qal?b ha, ancaq Qadrutun hesabat?nda isa lap ammal?-ba?l? duzal?? ediblar ha, v? vaxt?nda qabul edilmi? kimi d? sanadla?d?r?blar ha, rubluk mukafatlar?n? da veriblar ha. Biliram ki, boyunlar?na almayacaqlar ha, ondan sonra Qadruta bir teleqram vurar?q ha, onlar da hesabat?n giri?-�?x?? tarixini
  tasd?q edacaklar ha. Onda s?n bunlar? onlar?n g�zlar?na soxarsan v? ?u?an?n rubluk mukafat?n? da verdiracaksan, ya da Qadrutun mukafat?n? tutduracaqsan ha.\n Razmikin faktlarla dediyi, yoxlaman?n gedi?at?na k�m?yi olsa da, ?lqar?n aldad?c? h?r?k?tl?ri m?ni qane etmirdi. Guya  o, bu i?d? �z�n� d�zg�n,  h�quq?unas  t?siri ba???layan kimi g�st?rs? d?, doxsan d?r?c? bucaq alt?nda o t?r?f, bu t?r?f?  ?yil?r?k faktlar?n d�zg�nl�y�n�n s?n?dl??dirlm?sind? m?n? mane�ilik etm?y? �al???rd?. Bu is? ondan ir?li g?lirdi ki, o, m?nim, Razmikin diqt?sind?n faydaland???m? bilirdi.\n M?n, Razmikin dedikl?rini v? �z�m�n a?kar etdiyim n�qsanlar? t?sdiql?m?k ���n t??kilat?n ba? m�hasibini yan?ma �a??rtd?rd?m. Tipik hay qad?n? idi. Y?ni, sif?tind?n, duru?undan, boy-buxunundan, ir?li �?xm?? qarn?ndan tutmu? �z�n�n q?pq?rm?z? r?ngin? v? i�inin x?lt?na kimi hay qad?n? oldu?u anla??l?rd?. \n Ba? m�hasib bu y�nd?msiz b?d?niyl? art?q qar??mda idi
 . �ox da yax?? dan??a bilm?diyi t�rk l?hc?sind?: - e?idiram sizi, -dey?r?k, ?ll?rini qarn?n?n �st�nd? c�tl?yib m?ni s�zd�.\n M?n, bir xeyli qar??mdak? ka??z-ku?uzu o �z, bu �z�n? �evir?r?k, n?d?n ba?layaca??m? g�t�r-qoy etdim:\n M?n, - dey? bil?rsinizmi, ?u?a ?�b?sinin r�bl�k mukafat?n?  n? s?b?bd?n k?smisinz?\n Ba? m�hasib dili-doda?? ?s?-?s?, -onlar at�otu gecikdirmi?dil?r, ona g�r?.\n -Ne�? g�n gecikdirmi?dil?r? \n - Be? g�n.\n  - Hadrut ?�b?si d? hesabat? gecikdirib, �z� d? on be? g�n, b?s nec? olur ki, onlar?n pul mukafatlar? �d?nilib, k?silm?yib?\n - Yox, gecikm?yib ha, kim bunlar? deyib siza, -dey?r?k, sa??mda oturmu? Razmik? t?r?f �t?ri d? olsa q?yqac? n?z?r yetirdi.\n M?n, onun Razmik? t?r?f  k?s? bax??larla d�nm?sind?n  bel? ba?a d�?d�m ki, o,? Razmik? i?ar? etdi ki, adam ki?i olar, s?n �z haylar?n? bir t�rk? satm?san v? s?n biqeyr?ts?n, el? olmasayd?n �ox vaxt s?ni t�rk? ox?atmazd?lar? fikirl??dim v? on
 un �z�nd?ki yalan?n q?zart?lar?n? saf-��r�k ed?-ed?: -onda buyurun h?r iki ?�b?nin hesabatlar?n? g?tirin m?n?, -dedim.  \n O, sif?ti q?zarm?? halda  otaqdan �?xd?, el? sif?tinin p�rtm�?� getm?mi? d? hesabatlar ?lind? geri qay?td?.  M?n �z�m� o yer? qoymadan, ?vv?l �t?ri d? olsa, ?u?a ?�b?sinin hesabat?n? o �z-b� �z�n? �evir?r?k gecikm? tarixinin oldu?u kimi qald??? v? ?l d?ym?d?n  t?sdiql?ndiyini m�?yy?n etdim. Hadrut ?�b?sinin hesabat?n?n tarixinin ba??na it oyunu a�m??d?lar. El? bil yek? bir v?l il? x?rmanda var-g?l etmi?dil?r, hesabat?n tarixi q?rm?z? q?l?ml? kobud formada pozulmu? v? �st�nd?n laz?m olan tarix yaz?lm??d?. \n M?n �z�m� ona t?r?f tutaraq: - b?s buna n? deyirsiniz? B?s bu d�z?li? deyil, n?di? �z�n�z d? bu doyda t??kilat?n ba? m�hasibisiniz. M�hasibatl?qda is? bel? d�z?li?l?r etibars?z hesab edilir. Nadir hallarda edil?n d�z?li?l?r d? m�hasibatl???n t?l?b etdiyi qanunauy?un �?r�iv?d? apar?l
 an d�z?li?l?r hesab edil? bil?r ki, o da, riyazi hesablamalar zaman? mexaniki v? nadir s?hvl?rin d�z?li?in? aiddir. Y?ni, nadir bir s?hv r?q?min �st�nd?n q?rm?z? q?l?ml? nazik x?tt �?km?kl?, onun k?nar?nda d�z?li? etdiyiniz dig?r r?q?mi yaz?rs?n?z v? bunun da s?b?bi ham? ���n b?lli olur. Ancaq b�t�n bunlar?n hesabatlar?n t?qdim olunma tarixl?rin? he� bir aidiyy?ti yodur. Siz is? hesabatda sanki kotanla yer ?kmisiniz.\n Xank?ndind? yerl???n t??kilat?n t?fti?�isi Razmikin ?l alt?ndan m?n? �t�rd�y� teleqram?n cavab?na t?krar?n bird? baxaraq:  -al?n bax?n v? yalan dan??may?n, bu da Hadrutdan ald???m?z teleqramd?, dey?r?k, ?limd?ki teleqram? ona uzatd?m.\n   Onun sif?tind?ki p�rt�k q?zart?lar h?ddini �oxdan a?m??d? v? art?q onun hay olmas?n? isbat etm?y? ehtiyac yox idi.\n Qula?? s?sd?ymi?, n?ydis?, t??kilat?n m�diri, ya?da v? g�rk?md? ba? m�hasibind?n geri qalmayan Qreta Nikolayevna  i�?ri girdi v? n? ba? verdiyini anlamaq ist?di. Onsuz da
  bu qad?n?n ?ovnist oldu?u m?n? �oxdan b?lliydi v? onu g�r?n kimi, yad?ma d?hlizd?ki ??r?f l�hv?si d�?d�. H?r halda ??r?f  l�vh?si ??r?fsiz v? insanlar? aldadan bir n?sn? kimi beynimd?n ke�s? d?, bu Xank?ndind? yerl???n ??r?f l�hv?sind? hay ?�b?l?rinin qabaqc?l hay i?�il?rind?n ba?qa, ?u?a ?�b?sind? i?l?y?n n? az?rbaycanl? v? n? d? rus mill?tind?n ibar?t bir n?f?rin d? olsa ??kili yox idi, ancaq orda da i?l?y?n haylar?n ??kili var idi. Bu n?sn?l?r x?rda  olsa da k�n�l buland?ran  n?sn?l?r idi. \n Hiss etdim ki, Qreta da h?rif deyil v? i�imd?n ke�?nl?ri ba?a d�?�r v? hiyl?g?r dilini i?? salmaq ist?yir, m?n buna imkan verm?d?n,  - Qreta Nikolayevna, nec? ola bil?r ki, ?u?a ?�b?sind? ancaq haylar yax?? i?l?mi? hesab edilir, az?rbaycanl?lar, ruslar yox, ?g?r yax?? i?l?y?n varsa, niy? bu ??r?f l�hv?sind? onlar da olmas?n?\n Qreta Nikolayevna bozarm?? ??kild?, -yox, yox, niy?ki var, m?n onlar? Bak?dak? ??r?f l�hv?si ���n g�nd?rmi??m.\n M?n o
 nun n? dediyini g�yd? tutaraq, - yax?? el?misiniz, siz g?r?k onlar?, el? bax bu Xank?ndind? yerl???n t??kilat?n ??r?f  l�hv?sind? d? yerl??dir?rdiniz, d�nya da??lmazd? ki.\n O, susma?? il? ?m?ll?rini �rt-basd?r etm?y? �al???rd?, ancaq gec idi. Eyni zamanda tez olma??n?n, yaxud gec olma??n?n onun ���n he� bir f?rqi yox idi. O, �z mill?t�i v? ?ovnist i?inin ustas? idi. Ancaq o da var idi ki, o, yax?? anlay?rd? ki, m?n onlar?n dabbaqda g�nl?rin? b?l?d idim v? yeri g?l?nd? sif?tl?rin? �?rpma?? bacarsam da,  onlar?n da yeri g?l?nd?, �zl?rin?  m?xsus y�k g�t�m?k qabiliyy?tl?ri var idi v? bu, bizi aldadaraq, ba??m?z?n alt?na yast?q qoymalar?yd?. El?,  Qreta da g�z�m�n �n�nd? bir anl??a y�kl� qat?r? xat?rlatd?  v? bu y�k�n alt?ndan xilas olma?a �al???rd?.\n H?l? onun iqtisad�?s? ?nessan? demir?m. Bir biqeyr?t az?rbaycanl?n?n pullar?n? Moskvan?n ?n bahal? mehmanxanalar?nda x?rcl?dib, kef �?k?-�?k?, az?rbaycanl?lar?n ?lehin? dan??ma?
 ?ndan da qalm?rd?. Dan???rd?, ��r?ymizi yeyb, suymuzu i�?-i�?, dan???rd?, o papa?? ?lind? qalm???n pullar?n? g�y? sovura-sovura...\n Hirs m?ni alt-�st etmi?di, ��nki ���m�zd?n biri, it oynadan idi, o biri d? bunlara i?l?y?n idi, ?traf?mda da haylar. M?nim d? bunlarla bel?, t?limata uy?un r?ftar?m?n qar??s?nda h?r n?sn? ola bil?rdi. Onu da hiss edirdim ki, onlar m?ni milli m?s?l?y? y�nn?ndirm?kl? q?z??d?rmaq v? n?y?s? nail olmaq ist?yirdil?r. Ancaq Xank?ndi t??kilat?n?n m�diri Qreta  �z hay dilind? n? x?rdalad?sa, ?nessa otaqdan �?x?b getdi. Bu minvalla, Qreta xahi? etm?k ist?yirdi ki, n�qsanlar? akt material?nda g�st?rm?yim. Salm??d? hay bicliyin?, ��nki H?s?n Qas?movi�d?n qorxmasa da, �?kinirdi.\n M?n art?q bu n�qsanlar? oldu?u kimi akt material?nda s?ralam??d?m.  \n\n VII\n M�zakir? H?s?n Qas?movi�in yan?nda idi. Idar? heyy?tinin m�avinl?ri, ?�b? r?isl?ri v? Qreta ba?da olmaqla,  Xank?ndind?n g?lmi? n�may?nd?l?r d? kollegiya 
 stolinun ?traf?nda �z yerl?rini tutmu?dular. Xank?ndind?n g?lmi? n�may?nd?l?rin i?tirak?ndan bilm?k olurdu ki, bu m�zakir? onlara g�r?di, ancaq m�avinl?rin v? ?�b? r?isl?rinin i?tirak? onu ?sasland?r?rd? ki, bu m�zakir? t?kc? yoxlama material?na g�r? deyildi, h?m d? Xank?ndind? f?aliyy?t g�st?r?n t??kilat?n b�t�vl�kd? i? f?aliyy?tin? y�n?ldilmi? bir m�zakir? idi. Buna baxmayaraq, ilk ?vv?l Moskvadan daxil olmu?  ?ikay?t ?riz?si ?sas?nda apar?lm?? yoxlama i?inin n?tic?l?ri dinl?nildi v? bu y�nd? ba?qa ?�b?l?rl? ba?l? dig?r n�qsanlar?n s?ralanma?? da, ona qar?? iradlar?n bildirilm?si d? m�zakir? obyektin? �evrilmi?di. M�zakir?y? geni? formada ba?lan?lmas?nna baxmayaraq, haya yax?n, ona yalmanmaq ist?y?n ?�b? r?isl?ri n�qsan?n birini q?sqana-q?sqana deyirdis?, dig?rinin �st�nd?n ke�m?y? �al???rd? v? yaxud da s?ralad?qlar? n�qsanlar? yenid?n yum?altma?a �al???rd?lar. Bir s�zl? m�dafi? m�vqeyind?n �?x?? edirdil?r.\n M?n h?l?
  �?x?? etm?mi?dim. �?x?? ed?nl?rin ?sil simalar?n? m�?yy?n etm?k ���n, g�z�m onlar da, fikirim is? �z�md? idi. G�z g�r?-g�r? h?r n?sn? �rt-basd?r edilm?y? y�n?lmi?di. G�z g�r?-g�r? bu boyda n�qsanlara yol verilmi?, bu boyda milli ?ovinist �ns�rl?rin t�?yan etdiyi t??kilat?n f?aliyy?ti, he� n? olmam?? kimi d?y?rl?ndirilm?y? y�n?lmi?di. Bax, onda milli m?s?l? bizim ���n xo?a g?lm?y?n hal kimi qar??lana bil?rdi. Ancaq milli m?s?l?ni az?ri t�rkl?rinin �z�n? g�zg�r?si qabardan v? Respublika t??kilat?n?n tabeliyind? olan Xank?ndind? yerl???n bu t??kilat?n ?ovnist v? mill?t�i i?�il?rinin ba? idar?d? �ox sayda t?r?darlar? da �ox sayda tap?l?rd?. Bu da guya biz? tarix?n verilmi? b???riliyimiz v? insanl???m?zdan ir?li g?lirdi. V? n?y? g�r? bu b???rilik, bu insanl?q t?kc? m?n? aid olmal?yd?, bu torpaqda ya?ayan ba?qa birisin? yox.  Bax bunu fikirl???n d? adam ancaq �z i�ind? dan??a bilirdi, �z i�ind? d�?�n? b
 ilirdi, onu da qorxa-qorxa, gizl?y?-gizl?y?, y?ni onda yerin qulaqlar? insan qulaqlar?ndan daha ??kl?nmi? g�r�n�rd�, onda onlar ���n yox, yaln?z  m?nl?r v? bizl?r ���n yasaq olan milli m?s?l? h?r add?mda bizim ���n t?hl�k? obyekti idi. V? tarix?n biz? qar?? y�n?lmi? bu qeyri insani h?r?k?tl?r, t?kc? biz? qar?? qanunil??irdi. Bir anl?q  Bak? v? Xank?ndi aras?ndak?  sovet imperiyas?n?n ?ovinist siyas?ti  m?ni rahat buraxmad???ndan, �z�md?n as?l? olmayaraq aya?a qald?rd? v? s�z�m oldu?unu bildirm?k ���n H?s?n Qas?movi�in �z�n? baxd?m.\n O, - h?, e?idir?m sizi, -dey?r?k, m?n? i?ar? etdi.\n- H?s?n Qas?movi�, -dedim, -burda �?x?? ed?n yolda?lar Xank?ndind? ba? \nvermi? olaylara s?thi yana?d?lar v?  onu m�dafiy? m�vqeyind? durdular. Bu da ondan ir?li g?lir ki, onlar?n Xank?ndin? gedi?l?ri v? ?m?lli-ba?l? qar??lanmalar?, onlar? m?cbur edir ki, bel? bir ciddi m?s?l?nin m�zakir?sin? bu y�nd? d?, qeyri leqal yana?s?nlar. ?slind? Xan
 k?ndind? yerli hay ?halisi il?, yerli v? k�kl� Az?rbaycan ?halisind?n ibar?t i?�il?rin aras?nda �ox b�y�k f?rqli yana?ma m�vcuddu. Bu da Xank?ndind? yerl???n bizim t??kilat?n hay mill?tind?n olan r?hb?rinin apard??? qeyri sa?lam v? milli siyas?tin n?tic?l?ridi. Burda akt material?nda qeyd edilmi? n�qsanlarla, aidiyy?ti ?�b?l?r tan?? olmas?na baxmayaraq, h?min n�qsanlar?n �st�nd?n s�kutla ke�dil?r. Qeyd etm?k laz?md?r ki, b�t�n bunlar ondan x?b?r verir ki, ?�b? r?isl?ri Qreta Nikolayevnan?n qar??s?nda sanki g�zl?ri k�lg?lidi, yaxud da ondan �?kinirl?r. Q?bul olunmu?, Az?rbaycan ssr-nin konstutisiyas?na ?sas?n Az?rbaycan dili �mumxalq dili olmas?na baxmayaraq, Xank?ndind? respublika t??kilat?n?n ist?r �z�nd? v? ist?rs? d? tabe�iliyind? olan t?kilatlar?n �nvan g�st?ricil?ri hay v? rus dill?rind? f?aliyy?t g�st?rir. Muxtar Respublikan?n tabeliyind? olan ?u?a ?�b?sin? ayr? se�gilik h�km s�r�r.  Az?rbaycan ?halisin?n s?x ya?a
 d??? dig?r rayonlar da bir n?f?r d? olsun  az?rbaycanl? kadr  bizim sistem? i?? g�t�r�lm�r. M?n respublikam?za aid Xank?ndi t??kilat?n? bizim t??kilatdan t?crid olmu? formada g�r�r?m. V? b�t�n bunlar?, bayaq �?x?? ed?n ??xsl?r g�r?-g�r?,  hans? x�susiyy?tin? g�r? bu xan?m?n i? f?aliyy?ti q?na?tb?x? hesab edil?r?k,  d?y?rl?ndirilm?lidi?  Ona g�r? d? bu i?? ciddi yana??lmas?n? xahi? edir?m.\n H?s?n Qas?movi� diqq?tl? m?n? qulaq asd? v? ?s?bil??mi? halda yerind?n dik qalxd?:\n- M?n, h?l? bir ne�? il bundan ?vv?l Xank?ndind?  olanda demi?dim\nki, Qreta Nikolayevna bel? i?l?m?k olmaz, bu ?ovnistliyinizd?n, bu mill?t�iliyinizd?n ?l �?kin. M?nim m�avinl?rim v? ?�b? r?isl?rim d? yoxlama materiallar? il? tan?? olmalar?na v? ba? ver?n n�qsanlar?n m�vcudlu?unu g�r?-g�r?, bu ciddi m?s?l?nin m�zakir?sind?n yan ke�m?y? �al???rlar. G�r�n�r Qreta xan?m onlar?n �z�n? h?mi?? yax?? bax?b, onlar da h?r n?sn?nin �st�n� malalay?b, �r
 t�b ke�m?k fikirind?dil?r. ?O, m?n? i?ar? ed?r?k?, - m?n yolda??n fikir v? t?klifl?riyl? tamam ??rik?m v? Qreta Nikolayevnaya ciddi t�hm?t elan edilsin, �� ay vaxt verilm?kl?, t?fti? ?�b?sin? h?val? edilsin ki, bu verilmi? vaxt m�dd?tind? d�n�? yarad?lmazsa, Qreta Nikolayevan?n v? aidiyy?ti i?�il?rin i?d? qal?b, qalmamas? m?s?l?si m�?yy?n edilsin.\n Az da olsa, H?s?n Qas?movi�in bu �?x???ndan v? Qreta Nikolayevnaya veril?n ??idd?tli t�hm?td?n?  bir q?d?r rahatlansam da,he� bu da m?ni qane etmirdi. Ancaq hiss etdim ki, c�zi d? olsa,  bu ya?ad???m torpa?a, bu v?t?n? balaca bir i? g�r? bilmi?dim. ��nki d�nyan?n imperiya vaxt? ?limd?n bundan ba?qa bir n?sn? g?lm?diyini yax?? bilirdim. V? bu bax?mdan i�imd?n a??r bir y�k g�t�r�lm�?d� v? sar???n z?minin tax?l d?rzisi q?d?r y�ng�ll??mi?dim.\n Qreta Nikolayevna tipik haya b?nz?r x?lt? i�ind?, suyu s�z�lm�? formada, t?k-t?nha k��?l?rin biriyl? add?mlay?rd?, g�t�rd�y�
 �� y�n, hay mill?tind?n olan ?�b? r?isl?rind?n birinin evin? t?r?f  ald??? y�n idi. Fikirl??dim ki, bu da bir hiyl?di, hay mill?tind?n olan ?�b? r?isi �nc?d?n gedib v? bilir ki, Qreta onun evind? �ox oldu?undan evi yax??  tan?y?r v? �z� g?lib �?xacaq. �l�� c?zas?n? ?hat? ed?n ?mr? birinci d?rk?nar? m?n qoydu?umdan  �z�m� rahat hiss edirdim.  \n\n VIII\n\n T?fti? ?�b?sinin r?isini haylar ?baqaja? qoymu?dular. Guya  �z�n� apara bilmir, ?alka?d?? v? s. v? i.\n El? bu s?b?bd?n d? t?fti? ?�b?sin? t?z? r?is g?lmi?di, q?r?m?ndan bacar?ql? adama ox?ay?rd?. C�nki h?r i?in yerini-yata??n? bilirdi v? burda alver etm?k, bazarl?q yapmaq, r�?v?t d? yox deyilddi. H?tda pamb?qla ba? k?sm?k d? yox deyildi.\n Yuxar?larda  q?bul edilmi? q?rarlar?n n?tic?sind? pul d�vriyy?si yax??la?m??d?, t?kc? qal?rd? ki, onun �hd?sind?n ?layiqinc?? g?l?s?n, o da bu t?z? r?isin ?lind? ?semi�ka? �?tdamaq kimi bir n?sn?ydi.\n M?n aran rayonlar?n?n birind? ezamiyy?d? i
 dim. ?�b?d?,  Bak?dan kadrlar ?�b?sind?n z?ng edib m?ni axtard?qlar?n? bildirdil?r.    �mumiyy?tl?, ke�mi? ittifaq?n telefon ??b?k?si, ?sas?n sifari?li xarakter da??y?rd?. Avtomat telefon stansiyas? vasit?siyl? kadrlar ?�b?sin? z?ng etdim, salam-k?lamdan sonra, sabah tezd?n v?  t?cili Bak?da olma??m t?l?b edilirdi. S?h?r tezd?n, y?ni i? ba?layan vaxt? �z�m� kadrlar ?�b?sinin ota??na sald?m: - n? m?s?l?di, - dey?, narahat formada maraqland?m.\n- Biz d? bilmirik, sizi H?s?n Qas?movi� ota??nda g�zl?yir, -dey? kadrlar\n?�b?sinin r?isi bildirdi.\n S?drin yan?nda m?nd?n ba?qa t?fti? ?�b?sinin t?z? r?isi d? var idi. O, s?drin qar??s?nda �z�n? m?xsus g�rk?md? sakit v? dinm?z oturmu?du. M?n d? s?drin t?klifin? r??m?n stullar?n birind? ?yl??ib, n? s?b?b? �a??r?lma??m?n n?tic?sini g�zl?dim. Ancaq narahatl??a d?ym?diyini hiss etmi?dim v? bilirdim ki, h?r halda yax?? i? ���n �a??r?lm??d?m. El? olmas?n?, ondan hiss etdim ki, s?dr ?vv?l m?nim �z�m? v
 ? sonra da t?z? r?isin �z�n? xo? ehyamla baxd? v? t?fti? ?�b?sinin r?isin? yenid?n d�n?r?k s�z? ba?lad?:\n- Bu yolda?? y?qin ki, tan?y?rsan, i?l?diyi m�dd?td? �z�n� bacar?q?, \nt?cr�b?li v? savadl? bir kadr kimi g�st?rib, ?lav?d?, qeyd edim ki, sovet ittifaq? kommunist partiyas?n?n �zv�d�.\n Fikirl??dim ki, ax?, s?drin hesabat?n?n m?n? n? aidiyy?ti, ancaq fikirim-zikirim ondayd?. O, aram-aram, fasil? verm?d?n dan???rd?:\n- H?, m?n, onun kimliyi bar?d? ?trafl? m?lumat alm??am, v? g�r�r?m ki, \nl?yaq?tli v? bacar?ql? kadrd?r.\n Bu yerd? m?nim yad?ma d�?d� ki, t?fti? ?�b?sinin t?z? r?isini yax?ndan tan?y?ram, h?tda k�hn? r?isimin i?tirak? il? onunla nahar da etmi?ik. Eyni zamanda k�hn? r?isim t?r?find?n  bu adam?n yazd???, hans?sa, bir ?lyazmas?n?n eyb?c?r d�z?li?l?ri d?, bu d�z?li?l?rd?n �t�r� bir ne�? d?f?, onun yan?na g?lm?si d? yad?ma d�?d�.\n Bu fikirl?rin duman?nda aran? da?a, da?? arana da??d???m anda, H?s?n Qas?movi� y
 �ks?k v? r?hb?r tonunda m?n? t?r?f d�n?r?k, nitqini davam etdirdi:\n- M?n, t?fti? ?�b?sinin t?z? r?isi il? s?n g?lm?mi?d?n ?vv?l, s?nin bar?nd? \nbir xeyli fikir m�badil?si emi?ik v? bu t?klif d? m?nim ??xsi t?klifim olub. Q?raq bucaqdan t?klifl?r �ox idi, m?n is? bir t?klifin �z?rind? dayanas? oldum v? t?fti?in t?z? r?isi il? d? bu bar?d? raz?l??a g?l?si olduq. Bu t?klifin �z?rind? dayanma??m is? el? bel? deyil, bir s?ra i?l?rd? s?ni s?naqdan ke�irmi??m v? his etmi??m ki, t?miz adams?n?z, sistemin t?snifini he� kim? vermirsiniz v? sistemin m�dafi?sinin �hd?sind?n layiqinc? g?l? bilirsiniz, tamahs?z adams?n?z, insanlara qar?? davran???n?z m�sb?tdi, ��nki biz ?hali il? i?l?yirik, onlara qar?? qay?? bizim ?sas silah?m?z olmal?d?r.    B�t�n bunlar? n?z?r? al?b, sizi t?fti? ?�b?sinin r?isin? m�avin t?yin etm?k ist?yir?m, t?klifin n?dir.\n M?nim bu m?s?l?d? t?r?dd�t etm?yim, yaxud etm?m?yim n? ver?rdi ki, ax?, fikirl??dim v?, -m?sl?h?t sizindir, -dey
 ?r?k, yeni g?lmi? r?isin �z�n? baxd?m.\n O, daxili telefonu g�t�r�b kadrlar ?�b?sinin r?isin?, - Niyazovun ??xsi i?ini g?tirin, g?lin yan?ma, -dey?r?k telfonun d?st?yini g�t�rd�y� yer? d? qoydu.\n Kadrlar ?�b?sinin r?isi, ucaboylu, sar???n, sar???n da b??lar? olan orta ya?l? ki?i ?lind? d? m?nim ??xsi i?im i�?ri girdi v? h?min s?n?dl?ri s?drin stolunun �st�n? qoydu. \nS?dr m?nim ??xsi i?imi diqq?tl? v?r?ql?yib, kadrlar ?�b?sinin r?isin? i?ar?yl?, m?ni g�st?r?rk, -bu g�nd?n etibar?n bu yolda?? t?fti? ?�b?sin? r?is m�avini t?yin edir?m v? ?mrl??dirib t?sdiq ���n t?qdim edin, -dedi.\n Sovet t?hl�k?sizlik sisteminin k??fiyyat�?lar?na ox?ayan bu sar???n ki?i, - ba? �st? H?s?n Qas?movi�, -deyib, otaqdan �?xd?.\n M?n? g�st?rdiyi etimada g�r? H?s?n Qas?movi�? t???kk�r�m� bildirib, otaqdan �?xd?m v? t?fti? ?�b?sin? qay?d?b, stulumda ?yl??dim. I? yolda?lar?m bir az ?�bh?li-?�bh?li m?n? baxd?lar, m?n is? he� bir n?sn? 
 olmam?? kimi, t?fti?i i?inin apar?lmas?na aid t?limat qaydalar?n? v?r?ql?yib oxusam da, fikirim ayr? yerd? idi. Bir az ke�m?di ki, t?fti? ?�b?sinin t?z? t?yin edilmi? r?isi d? ota?a daxil oldu, c?ld v? ??stli ad?mlarla ir?lil?yib �z yerind? oturdu. Bel? fikirl??m?k olard? ki, v?zif? adamlar?n?n ham?s?nda bu x?st?lik ?n?n?vi xarakter da??y?rd?. B?z?n �zl?rini ?trafdan t?crid olmu? kimi v? yaxud t?k?bb�rl� g�st?r?n, �z�n? s??mayan, h?tda bel? al�aq da?lar? yaratm?? kimi, t?sir ba???layan r?isl?r? d? rast g?lm?k olard?. Hiss etm?k yerin? d�?�rd� ki, bu t?z? g?lmi? r?isin d? iddialar? balaca deyildi, �z�nd?n b�y�k idi. Bunu onun davran???na ?sas?n d? m�?yy?n etm?k olard?.\n T?z? r?is, ani bir s�kutdan sonra �z�n� t?fti? ?�b?sinin t?fti?�il?rin? tutub,\n - bilirsiniz, s?drimiz H?s?n Qas?movi� m?ni yan?na �a??rm??d?, bir xeyli s�hb?t etdik v?  m�avin m?s?l?sind? h?r ikimizin fikirl?rimiz v? t?kilifl?rimiz �st-�st? d�?d�, art?
 q bu g�nd?n etibar?n yolda? Niyazov  m?nim m�avinimdir, t?limat qaydalar?na aid n? sual?n?z olsa onunla h?ll ed? bil?rsiniz, i? bar?d? m?s?l?l?ri m?niml? b�l�?d�r?c?ksiniz.\n Ir?li �?kilm?yim? g�r? i�imd? sevinc v? q�rur hissi ke�irs?m d?, h?lli m?nmlik olan m?s?l?l?rin h?ll edilm?sind? m�st?qil olmama??m?n a?r?lar?n? indid?n anlay?rd?m.   Onu da yax?? anlay?rd?m ki, bu ir?li �?kilm?yim d? r?mzi xarakter da??y?rd?. ��nki ist?r s?dr v? ist?rs? d? ?�b? r?isi yax?? bilirdi ki, kimi hans? v?zif?y? t?yin etm?k  g?r?klidir. Demaqoq olmas?n, ortada olan? orda-burda dan??mas?n, tamahkir olmas?n, ortadak? g?lirl?rd? g�z� olmas?n, dindir?nd? dinsin, dindirm?y?nd? d? lal-dinm?z yerind? otursun. Dem?li m?nim robotdu?um daha m?sl?h?tliymi? ki, m?ni d? ona m�avin t?yin etmi?dil?r. ?g?r ke�mi? r?isin yerin? ?bacar?ql?? v? ?qurd kimi? da??d?c? bir adam g?tirmi?dil?rs?, onda ona m�avin el? adam olmal?yd? ki, onun i?in? burnunu soxmas?n, tikans?z olsun, onun ci
 ddi �at??mamazl?qlar? olsa da bel?, g�z�n� yumsun. Yoxsa ke�mi? m�avin, M�rs?l kimi, burnunu tez-tez, n?in ki, r?isin i?-g�c�n? soxurdu, h?tda ne�? i? yolda??n? orqan i?�il?rinin ?li il? l?k?l?yib s?radan  �?xartm??d?.  N?di-n?di, yoxlama zaman? t?fti?�i, filan s?n?dd? bu n�qsan? g�r? bil?rdi v? yaxud g�rm?y? d? bil?rdi v? h?r n?sn?y? ehtimal g�z�yl? baxan t?fti? ?�b?sinin r?is m�avini M�rs?l, bu minvalla �oxlar?n?n dal?ndan d?ydirmi?di, �z� d? h?r?nin belind? bir iz buraxa-buraxa. Bax onda m?nim yad?ma bank S�hb?tin qarda?? Abdulla m�?llim d�?m�?d�. Ehtimal n?z?riyy?si m�?llimi. O, adi misallarla bu f?nnin maraql? t?r?fl?rinin d? ehtimal oldu?unu izah ed?-ed?, bizi ehtimallara k�kl?y?rdi. Ancaq Abdulla m�?llimin ehtimal?ndan  f?rqli olaraq M�rs?lin ehtimal? qorxu hissind?n yaranm??  ehtimallardan biriydi, y?ni ?g?r orqan i?�il?rind?n biri onun �st�n? q??q?rsayd? ki, bu s?n?d d? bu n�qsan? g�rm?k olard?, ya 
 yox, o da a??na-qaras?na fikir verm?d?n dey?c?kdi ki, ?b?li yolda? hakim g�rm?kd? olar, g�rm?m?k d??. Bu da o s?n?dl?ri ara?d?ran v? M�rs?ll? birg? f?aliyy?t g�st?r?n i?�il?r? baha ba?a g?l?c?kdi. Bax, yolda?l?qda M�rs?l, yolda?l??? yox, �z�n�n can qorxusunu m�dafi? ed?c?kdi. V? onun bu can qorxusu is? onlara baha ba?a g?l?c?kdi, ?m?k kitab�alr? korlanacaqd?, i? yerl?rini itir?c?kdi, t?z? i? yerl?rin? g�t�r�lm?y?c?kdi v? s.\n Fikir m?ni aranl?-da?l? g�t�rd�y�nd?n i? yolda?lar?m?n nahar fasil?sin? getm?l?rini v? nahar fasil?sinin olma??n? bel? unutmu?dum.  Hiss etdiyim o olmu?du ki, otaqda r?isl? m?n ba?a-ba?a qalm??d?m. B?lk? d?  r?is dem?s?ydi ki, nahar fasil?sin? niy? getm?mis?n, el? ?limd? olan bu t?limat b�t�n g�n�m� b�sb�t�n ?limd?n alacaqd?. O, f�rs?td?n istifad? edib, m?niml? i?in gedi?at?n?, rayon ?�b?l?rinin hans? yax??, hans? pis i?l?m?sini v? s. ayd?nla?d?rmaq ist?di.\n O, divardak? mar?urut x?rit?sin? yax?nla?d?, sa
 ? ?lind?ki �ubu?u x?rit?nin �st�nd? g?zdirdi. Bir-iki ya?l? zonan? m?n? g�st?rdi v? soru?du: -bu zonalar nec?dir, olmusanm? bu zonalarda, -dedi.\n Onun n? dem?k ist?diyini alm??d?m, el? ona g�r? d?, -yox, -dedim.\n O, ?lind?ki �ubu?u x?rit?d? g?zdir?-g?zdir?, - e?itmi??m ki, bu zonalar yax?? pul ver?n zonalard?. S?ni g�nd?r?c?m bu zonalara, ��d?n birin?.\n M?n o yax?? zonalar? ba?a d�?s?m d? he� birind? olmam??d?m v? el? bu s?b?bd?n d? bu ��d?n biri m?s?l?si m?ni ya bil?r?kd?n, ya da bilm?y?r?kd?n almad? v? soru?dum, - r?is bu ��d?n birini ba?a d�?m?dim, bu n? dey?n n?sn?dir?\n O, bic-bic g�l�ms?di v? k��?y? baxan p?nc?r?y? t?r?f ba??n? qovzaya-qovzaya, - s?n indiy? kimi i?l?yirs?n, bilmirs?n ��d?n biri n?di? Y?ni s?ni ?n yax?? zonalara g�nd?r?c?m, ald???n r�?v?tin ikisi s?nin olacaq, biri is? m?nim.\n M?n fikirl??m?d?n k?s? cavab verdim, - r?is m?ni ??n pis? rayonlara g�nd?r, - dedim, ��nki ??n pis? rayonlar, kas?b, y?ni g?liri 
 a?a?? olan v? bunun m�qabilind? d? pul verm?y? imkanlar? olmayan rayonlar idi.\n O,  bic-bic �z�m? baxaraq, -m?nim ���n f?rqi yodu, m?n s?nin ���n deyirdim. Fikirl??dim ki, s?nin d? ail?n, u?a??n var, -dedi.\n M?n k?s? cavab verdim, - a?r?maz ba??m? niy? a?r?d?m ki, -dey?, fikirimi q?til??dirdim v? unutmu? oldu?um nahar fasil?sin? yolland?m.\n El? h?min g�nd?n d? m?nim taleyim? bir a??rl?q d�?d�, ancaq m?n buna pe?man deyildim. T?kc? pe?man�?l?q ondan ibar?t idi ki, r?is art?q m?ni incitm?y? ba?lam??d?. Bu incitm?yin s?b?bl?ri m�xt?lifdi, adam-adam? ist?m?s? hans? yolla ist?s? bezdir? bil?r, can?n? bo?az?na y??a bil?r. O, art?q bu minvalla da ba?lam??d? m?ni tez-tez, dabanbasd? ezamiyy?l?r? g�nd?rm?y?, aran v? da? rayonlar?na m�vs�md?n as?l? olaraq, qrafik tutulmas?na baxmayaraq, q???n q?? vaxt? da? rayonlar?na, istinin isti vaxt? is? aran rayonlar?na g�nd?rm?y?. O q?d?r tez-tez g�nd?rirdi ki, bir g�n qap?n? d�y?nd? b�y�k q?z?m qap?n?n g
 �zl�y�nd?n bax?b anas?na demi?di ki,? hans?sa ?mi? var qap?n?n �n�nd?, anas? da qap?n? a�?b, -q?z?m ?mi deyil, atand?, -demi?di.\n O q?d?r bezmi?dim, �al???rd?m ki, i? yerimi d?yi?im, r?isim bu i?d? d? m?n? mane�ilik edirdi. Bunlar? t?fti? ?�b?sin? daxil olan telefon z?ngl?rind?n hiss edirdim, ��nki otaqda ??h?r telefonu bir d?n? idi o da r?isin stolunun �st�nd? idi. B�t�n bunlar? telefon z?ngl?rin? ?ifahi v? m?xvi veril?n m?lumatlardan hiss edirdim. Eyni zamanda bunu m?n, t?z? i? yerl?rind? ilk ?vv?l, m?ni iki ?lli qar??lay?b, n�vb?ti  g�n is?? m�mk�n olmad?? v? yaxud ?siz? aid bizd? i? olmad????n? bildirirdil?r. Ayd?n olurdu ki, r?is m?ni ?m?lli-ba?l? m?nfi xarakteristikas? il? h?r n?sn?d?n uzaqla?d?r?rm??. \n\n IX\n\nYen? d?, aran b�lg?l?rind?n birind? ezamiyy?td? idim. Bak?dan z?ng etmi?dil?r. Yen? H?s?n Qas?movi�in m?ni axtard???n? x?b?rdar etdil?r. Ad?ti �zr?, ezamiyy?d?n qay?td???m g�n�n s?h?ri i?? �?xd?m. \nBirba?a s?drin yan
 ?na getdim. H?s?n Qas?movi� �z�n� m?n? tutaraq: - bilirsiniz,? t??kilatlarla i?l?y?n ?�b??nin r?isini i?d?n azad etdim, i?l?y?n deyildi. ?�b?nin i?i getmirdi, m�asir texnoloji prosesl?rl? ayaqla?a bilmirdi. Nazirlik v? ba? idar?l?rl?, el?c? d? yerl?rd? aidiyy?ti struktur b�lm?l?ri il? apar?lan i?l?r yar?tmazd?. ?limizd? r?hb?r tuta bildiyimiz bir �ox s?n?dl?r olmas?na baxmayaraq, yararlana bilm?dil?r v? bu imkanlardan istifad? ed? bilm?dil?r. Halbu ki, Az?rbaycan m?rk?zi komit?sinin,  bizim t??kilat?n inki?af?na y�n?ldilmi?, b�ronun q?rar? var. Ssri nazirl?r sovetinin, Az?rbaycan nazirl?r sovetinin, ssri d�vl?t bank?n?n, el?c? d? bizim Moskvadak? ba? idar?mizl? ssrinin b�t�n nazirlikl?ri v? ba? idar?l?ri il? birg? g�st?ri?l?ri m�vcuddur, �ox t??ss�f ki, ad? �?kil?n ?�b? yerind? say?r v? bu m?s?l?l?rin h?llini t??kil ed? bilmirl?r. Bizim m?qs?dimiz ?hali aras?nda  m?d?ni xidm?t sferas?n? genil?ndirm?k, y�ks?k s?viyy?d? xidm?t i?i yaratmaqd?r.
  M?n ne�? g�nd�r ki, bu bar?d? g�t�r-qoy etmi??m. D�zd�, bir ne�? n?f?r �st�m? minn?t�i sald?lar, yuxar?lardan z?ng etdirdil?r, o ?�b? onlar?n yeri deyil, o ?�b?d?, g?r?k ?lini �?rmalay?b, ?m?lli-ba?l? i?l?y?s?n. Odur ki, o ?�b?nin aya?a qald?r?lmas?n? s?nin i? qabiliyy?tinl?, bacar???nla v? savad?nla ba?l? oldu?unu fikirl??dim. H?r  hans? bir �?tinliyin  olsa, o m?s?l?l?rin h?llind? siz? k�m?k  edil?c?kdir. ??xs?n m?n �z�m o ?�b?nin i?ini n?zar?t? g�t�rm�??m, h?r hans? nazirlik v? yaxud yuxar? dair?d?n mane�ilik olarsa, m?ni m?lumatland?rarsan v? vaxt?nda onlar?n qar??s?n? alar?q. Odur ki, s?n?dl?rini haz?rla Moskvaya getm?lis?n, ��nki ?�b? r?isl?ri Moskvan?n ?tat vahidl?ridi. ?�b? r?isl?rinin i?? g�t�r�lm?si onlar?n q?bul etdiyi q?rara ?sas?n h?yata ke�irilir. M?s?l? art?q onlarla raz?la?d?r?l?b v? s?nin h?min ?�b?y? r?is t?yin edilm?yini  m?sl?h?t g�rm�??m. \n -H?s?n Qas?movi? h?r halda o ?�b?d? insititutdan 
 t?yinatla g?lmi? bacar?ql?, savadl? v? perspektivli g?nc kadrlar da olmal?d? ax?, -dedim.\n - El? onlar?n i?i m?ni qane ets?ydi s?h?rd?n s?ninl? bir quranl?q dan???q aparmazd?m. S?h?r getm?lis?n. Yax?nla? kadrlar ?�b?sin?, hans? s?n?d laz?msa,  haz?r el? v? Bak?-Moskva  reysin?, s?h?r u�u?u ���n indid?n bilet al. Ssri �zr? ba? idar?nin s?dri x?st?dir, i?? h?l?lik �?xm?r, s?ni birinci m�avin q?bul ed?c?k, �z�n� savads?z g�st?rm?, s?n? veril?n suallar i?imizin sah?l?rin? aid olacaq, t?l?sm?, ?vv?lc?d?n fikirl??, sonra cavabland?r.  \n T???kk�r edib otaqdan �?xd?m. Kadrlar ?�b?sin? yax?nla?d?m, qap?n? a�?rd?m ki, ke�mi? ittifaq?n t?hl�k?sizilik sisteminin polkovnikin? ox?ayan kadrlar �zr? r?is m?niml? qar??la?d? v? kobud sur?td?, -s?n d�n?nd?n burda olmal?yd?n, niy? gecikmisiniz, - dey? m?ni m?z?mm?tl?di.\n I?? ke�m?dimmi, �z-�z�m? gleyl?n?r?k, -ax?, siz m?n? d�n?n x?b?r etmisiniz, m?n d? d�n?n qay?td?m v? bu g�nd? i?d? oldum, h?m d
 ? el? ezamiyy? m�d?ti d? d�n?n qurtard?. El? s?drin yan?ndan �?x?ram, bilm?liydim ki, s?dr m?ni niy? axtar?b, bel? ba?a d�?m�?d�m ki, m?s?l? yoxlamayla, n? is? bir �at??mayan bir n?sn?yl? ba?l?d?. Ayd?n oldu ki, o, m?ni ba?qa bir m?s?l? il? yan?na �a??r?bd?.\n O, - yax??-yax??, ke� otur, g?lir?m, - dey?r?k, otaqdan �?xd?.\n Ota?a daxil oldum v? onun m�avini zarafat�?l oldu?undan bir az m?z? q?ld?q. M�avin Moskvada oldu?undan, bizim ordak? ba? idar?mizd? tan??? bir xan?mla s?rg�z??tl?rind?n dan??d?  v? salam?n? ona �ad?rma??m? xahi? etdi. M?n d? eyni ?hval-ruhiyy?d? onu arxay?n sald?m. Kadrlar?n ?�b? r?isi is? ?lind? apard??? qovluqla da geri qay?td?. M?n laz?m? s?n?dl?ri doldurdum. T?kc? partiya t??kilat?n?n xasiyy?nam?si qalm??d?. O da yaz?lm??d?, ?�b? r?isi doda??n?n alt?nda m?z?ldaya-m?z?ldya m?n? uzadaraq: - t?cili imzalat, g?tir m�h�r vuraq,- dedi.\n Otaqdan �?xd?m, partiya t??kilat katibinin ota??na daxil olan da, tan?d???m biriyl? hans?
 sa m?s?l?ni m�zakir? edirdil?r, m?ni g�r?n kimi, - i?in t?cilidi? - dey?, m?n? diqq?t k?sdi.\n- B?li, dedim, partiya t??kilat?n?n xasiyy?tnam?si yaz?l?b, ancaq \nimzalanmay?b, t?cili imzalanmal?d? v? m�h�r vurulmal?d?, s?h?r Moskvaya u�mal?yam, t?sdiq ���n.  \n O, - c?ld ol, c?ld ol, i?imiz �oxdu, -deyib, xasiyy?tnam?ni �z d?sti-x?ttil? imzalad?\n M?n otaqdan �?xd?m v? t?l?sik, yenid?n kadrlar ?�b?sin? yax?nla?d?m, s?n?d m�h�rl?ndi, indi yubanmadan  hava yollar?n?n bilet kassas?na t?l?sm?yim qal?rd?, h?l? bazara da d?ym?liydim. Laz?m? s?n?dl?r qoltu?umda, birba?a bilet kassas?na yolland?m, h?l? g�norta�st�n? xeyli zaman var idi. �st?lik bazara da vaxt qal?rd?, ?libo? at?la-at?la getm?yin d? l?zz?ti yox idi. Bilirdim ki, moskval?lar Az?rbaycan?n nar?n? v? konyak?n? �ox sevirl?r. \n Art?q bilet  kassas?nda idim v? xo?b?xtlikd?n adam az idi,  bilet d? var idi. Bilet ald?m v? s?n?dl?rin yan?na qoydum, fikirl??dim ki, ev? gets?m gec olacaq, yax??s? b
 udur birba?a bazara gedim. Taksil?r d? m?ni g�rm�?d�, biri d? g�z? d?ymirdi, yan?mda k�hn?-salxaq, nuh ?yyam?ndan qalma bir moskvi� dayand?, mindim v? s�r�c�d?n xahi? etdim ki, t?z? bazara s�rs�n v? be?-on d?qiq? d? m?ni g�zl?sin. Onsuz da bazardan nardan ba?qa bir n?sn? almal? deyildim. Ald???m be?-on kilo narla tez-t?l?sik ma??na qay?td?m. Qal?rd? konyak m?s?l?si, konyak alsayd?m, pul �?tinliyim olacaqd?, almasayd?m da,  nar h?diyy?l?rimi konyaks?z verm?yim �?tin olacaqd?. F�rs?td?n istifad? edib, ??rab zavodunun yan?nda f?aliyy?t g�st?r?n ?�b?nin m�dirin? z?ng etdim. H?r�?nd ki, onun m?nd?n xo?u g?lmirdi. ��nki i?d? buraxd??? n�qsanlara g�r? hay mill?tind?n v? apar?c? i?�il?rind?n birini i?d?n �?xartd?rmaq ist?yirdim, el? h?min ?�b?nin m�dirinin adam? oldu?una g�r?, yalvar-yaxardan sonra t�hm?tl? kifay?tl?nmi?dik.\n Z?ng ed?r?k, ?�b? m�dirini ad?yla �a??rd?m, telofonun o ba??ndan, -b?li, b?li e�idir?m, sizi, - dey?
 r?k, m?ni dinl?m?y? �al??d?, �z�m� t?qdim ed?r?k, - siz? i?im d�?�b, -dedim.\n Sakit-sakit, asta v? aramla, - e?idir?m, - dedi.\n -m?n? t?cili 20 ?�?? t?miz zavod m?hsulu olan konyak laz?md?r,- dedim.\n Telefonun o biri ba??ndan, -n? vaxta?\n -ax?am �st�n? kimi, ��nki s?h?r reysi il? Moskvaya u�mal?yam.\n -bir g�n ?vv?l dey?ydin, - dey?, m?z?ldand?.\n Ona  izah etm?y? �al??d?m, - m?n �z�m d? he� bilm?mi??m, ezamiyy?td? idim, d�n?n x?b?r edibl?r, bu g�n d? g?lib s?n?dl?ri yan-y�r? etmi??m, ma?azaya getm?y? vaxt olsa da, onlar?n konyaklar?na bel ba?laya bilmir?m, h?r-halda onlar sizin xidm?tinizd?dirl?r. Bilir?m ki, siz? keyfiyy?tli m?hsul ver?rl?r. Bizim Moskvadak? ba? idar?mizin bir ne�? i?�isin? h?diyy? etm?k ist?dim, -dey?, onu ba?a salma?a �al??d?m.\nBir saatdan sonra z?ng etm?yimi bidirdi. Bir saat �ox gec g?lib ke�di, ??rin qar??mas?na az qal?rd?. Bu, m?nd?n as?l? bir n?sn? deyildi, zaman dar?xd?r?c? idi.\nTelefon d?sd?yini g�
 t�r�b, bir d? z?ng etdim, katib?siydi, be? d?qiq?d?n sonra z?ng etm?yimi bildirdi. Be? d?qiq?d?n sonra yenid?n z?ng etdim, indi d? dedil?r ki, m�dir harasa �?x?b. Bir az h�vs?l?mi basd?m. Ba?qa n? etm?liydim ki, o q?d?r cibimd? pul olsayd?, m?n bel? har?n adamlara a??z a�ard?mm?, dey? �z-�z�m? var-g?l etdim. Yenid?n z?ng etm?kd?n ba?qa �ar?m qalm?rd?, telefonun d?sd?yini g�t�rd�m, k�n�ls�z-k�n�ls�z v? a??r-a??r n�mr?ni nec? y??m??d?msa, he� �z�md? hiss etm?mi?dim. Bu d?f? katib? m?ni �ox g�zl?tm?di, -indi qo?uram, dan???n,\n -dedi.\n Telefonun o biri ba??ndan m�dirin s?si g?ldi, - dediyin n?sn?l?r haz?rd?, hara g�nd?rim, -dedi.\n -i?d? g�zl?yir?m, bura g�nd?rs?n yax?? olar, el? buradan da ev? g�t�rr?m v? qavqarram, - dedim.\n Bu i? d? bel? d�z?ldiyind?n ba?qa bir i?im qalmam??d?, ax?amdan h?r n?sn?ni d�zd�m qo?dum v? s?brsizlikl? s?h?rin a�?lmas?n? g�zl?dim. Bilirdim ki, tezd?n bir q?d?r taksi �at??mamazl??? o
 lur, bir az ona g�r? narahat idim. \n S?h?r o ba?dan durmu?dum. �z�m� tara? etdim, b?n�v??yi k�yn?yimin yaxas?na ona uy?un qalstuk tapd?m, qaraya �alan kostumumu geyindim, ?limd?ki, �ox da iri olmayan y�k�ml? taksi dayanaca??na add?mlad?m. Taksi dayanaca??n?n da ad? var idi, ?ks?r vaxtlarda orda ara taksil?ri dayan?rd?, indi he� onlar da yox idi. G�zl?y?si oldum, ancaq hava liman?na gecikm?yim? g�r? dar?x?rd?m. Anca el? bil hiss edir?mmi? kimi, hardansa, k�hn? bir g�y volqa peyda oldu, qaz 24-l?r moda mins? d?, bu k�hn? qaz 21-l?r �z yerini h?l? bu t?z? ma??nlara verm?mi?di v? tank kimi, ora-bura ?�t�y�rd�. Ma??n yan?mda durma??yla, sor?u-suala tutmadan minm?yim bir oldu. ��nki �nvan? ?vv?l dem?kd?n �?kindim, fikirl??dim ki, bird?n uzaq yola getm?kd?n ehtiyat ed?r. S�r�c� o q?d?r d? pis adama ox?am?rd?, min?nd?n sonra ged?c?yim �nvan? bildirdim. \n Hava liman?na �atan kimi, tez qeydiyyat m?s?l?sini h?ll etdim, y�kl?rimin 
 d? baqajda yox, ?limd? aparaca??m? bildirdim v?  m�?yy?n s?b?b? g�r? ?limd? aparma??n icaz?sini d? ald?m. \n A??r y�kl? birlikd? s?rni?inl?r? qar??m??d?m, g�zl?m? zal?nda g�zl?m?yimiz uzun �?km?di, art?q daxili avtobusa minik ba?lam??d?. Bir t?h?r avtobusda �z yerimi tutdum v? Moskvaya y�n alas? u�a??n yan?nda idik. Biletim ikinci salona idi, ?vv?l ?limd?ki y�k� oturaca??n alt?nda, �?tinlikl? d? olsa yerl??dirdim. B?l?d�inin x?b?rdarl??? g?lm?d?n ehtiyat k?m?rini belim? vuraraq, yerimi m�hk?ml?ndirdim. ?slind? k?m?rin uclar?n? bir-birinin �st�n? qoymu?dum, �z�m� �idarlamaqdan z?hl?m gedirdi v? bel?lik?, sanki b?l?d�inin g�z�nd?n p?rd? asm??d?m.\n U�a??n m�h?rrikinin s?si g?ls? d?, h?l? yerd?ydik, bu, u�a??n s?maya qalxmam??dan ?vv?l yar?m saatl?q haz?rl?q v? nasas?zl???n?n saf-��r�k edilm?sin? ayr?lm?? zaman vaxt?yd?. Laz?m olan vaxt m�d?tind?n sonra, mavi r?ngd? pilot paltar? geyinmi? b?l?d�i peyda oldu, mar?urutu, �at
 as? zaman vaxt?n? elan etdi, ancaq k?m?rl?rin bel? b?rkidilm?sini d? unutmad?. U�aqdak? yerim xo?b?xtlikd?n, p?nc?r?nin yan?na d�?m�?d�, getdikc? yerd?n uzaqla??rd?q, s?man?n yuxar? qat? buludlarlanan ?h?t? edilmi?di. Art?q s?mayla buludlar?n aras?ndayd?q, buludlardan a?a?? he� n? g�r�nm�rd�. Sanki d?niz dal?alar? �st�nd? g?mimiz �z qanadlar?n? g?rib arxay?n v? y�ks?k s�r?tl? ir?lil?yirdi. Yan?mdak? oturacaqda oturan xarici adam t?siri ba???lad?, ?vv?l koreyal? kimi t?s?vv�r etdim, ancaq yapon oldu?u m?lum oldu. T?sad�f�, �z�ml? g�t�rd�y�m Yapon klassikl?rind?n Az?rbaycan dilin? t?rc�m? edilmi? ?eirl?r kitab? d�?d�. Tez d? �?xard?b ona g�st?rdim, Az?rbaycan dilini bilm?s? d?,  Yapon xalq?n?n klassik ?airl?rini kitabda verilmi?,  ??kill?rind?n tan?d? v? rus dilind? h?r biri haqq?nda maraql? m?lumat verdi. M?n yapon dilini bilm?s?m d? onun ?lind?ki yapon dilind? olan kitab m?ni maraqland?rd?,  ist?dim  m?nd? olan kitab? onun kita
 b?yla d?yi?m?k yolu il? ki�ik v? m?n?vi bir bazarl?q yapam. Yapon h?mkar?m buna raz? olmad? v? onda �z sad?l�hvl�y�m�z? ac???m tutdu v? yaponlar?n bu m�r?kk?b xarakterin? h?s?d apard?m. Ba??m?z s�hb?t? qar??d???ndan, b?l?d�inin, - k?m?rl?rinizi b?rkidin, be? d?qiq?dn sonra ?Domededova? hava liman?na enirik, -dem?si, m?ni yapon h?mkar?mla ?irin s�hb?td?n ay?rd?.\n Art?q u�a??m?z hava liman?nda yer? enmi?di, m?nim baqaj?m �z�md? oldu?una g�r? baqaj g�zl?m?y? vaxt itirm?y?c?kdim v? taksi dayanaca??na t?l?sdim. Cavan s�r�c�l?rd?n g�z�m su i�mirdi, ona g�r? d? ya?l? bir s�r�c�n�n taksisin? ?yl??dim, ged?c?yim �nvan? dedim. Yol boyu Moskvan?n pay?z f?slinin ?srar?ngiz g�z?lliyin? varm??d?m v? bu g�z?lliyin i�ind? bir u?aql???m gizl?nib qalm??d? sanki. Qal?n ?am a?aclar?n?n, �inar a?aclar?n?n aras?ndan uzanan yolda ?�t�y?n taksinin p?nc?r?sind?n g�c�m seyr etm?y? �at?rd? v? m?n d? bu f�rs?ti ?ld?n vermirdim. Bu an ya
 ?l? s�r�c�n�n sual? m?ni diksindirdi:\n -s?nok otkuda?\n - iz Baku.\n - oy, va?e Qeydar Aliyevi� koqo-nibud v?z?vaet sebya, oni svoim semyam pro??ayut.\n - a po�emu tak, - dey?, soru?dum,\n - oni boyitsa, potomu �to Qeydar Aliyevi� o�en stroq?y �elovek i o�en trebovatelnoy.\n M?n bu s�hb?td?n q�rur hissi ke�ir?r?k s�r�c�y?: - da, on o�en stroq?y i  velikolepniy �elovek, -dey?r?k, H. ?liyevin Az?rbaycan M?rk?zi Komit?sinin birinci katibi i?l?diyi m�dd?td? g�rd�y� i?l?rd?n dan??d?m.\n Me?? a?acla?n?n yarpaqlar? q?z?l? r?ng? �al?rd?, m?n bu �alarlar? u?aql???mda G?d?b?yin Q?xmalaxl? yayla??nda, ?vanovka k?ndinin �st yamaclar?ndak? me??l?rd?, Tovuzun Qarada?-B�y�k Q??laq istiqam?tind?ki me??l?rd? �ox ?ahidi olmu?dum. K?ndimizd?ki m?hl?mizin q?ra??ndak? k�knar a?aclar?n?n pay?z havas? is? h?mi?? m?niml?ydi. \n �atd???m �nvanda s�r�c�n�n haqq-hesab?n? ��r�d�b, sa?olla?d?m v? gec?ni pis ke�irm?dim, h?r n?sn? 
 �r?yimc? idi v? rahatland?m, ancaq bir narahatl???m var idi. Bu h?diyy?l?ri b�l�b, ayr?-ayr? torbalara qoymal?yd?m. Onda  indiki g�r�nm?y?n torba, �anta bollu?u yox idi, ancaq ka??z torbalardan istifad? ed? bil?rdim. Yax?? ki, orda i?l?y?n Valeri Yuryevi�i tan?y?rd?m v? narahat�?l??a bir ?sas olmad???n? fikirl??dim.\n S?h?r erk?n oyand?m, h?l? dan �z� yenic?  qara ilm?l?rini a?  ilm?l?rd?n saf-��r�k edib, aram-aram d?nl?yirdi. Havada ya??? iyisi yox idi, hiss olunurdu ki, pay?z g�n??i �z sar???n sa�aqlar?n? Moskvan?n k�knar me??l?rin? s?rgi ed?c?k. Bu ab-havan?n alt?nda qald???m evin eyvan?na �?xaraq, evin qar??s?ndak? park?n yarpaqlar? q?z?la �alan a?aclar?n?n ?trini, ?limd?ki siqaretl? birg? ciy?rl?rim? �?k?si oldum. Havan?n tez i??qlanmas?n? ist?yirdim, ancaq bu qara ilm?l?r, ilahi q�vv? t?r?find?n nec? toxunmu?dusa, el?c? d? qalm??d?, h?rd?n bir bir ilm? boyu a?ar?rd?. I�?ri ke�ib �ay-��r?k i?in? baxd?m, ?sas?n s?h?r yem?yimin 
 xobbisi bir tik? pendir, bir tik? d? ya? olmas?n? fikirl??ib, soyuducudan g�t�rd�m v? ruslar demi? ?zavtrak?m?? i?tahla yerin? yetirdim. Yem?kd?n sonra eyvana bir d? �?xd?m, bir siqaret yand?rd?m, art?q adamlar?n yava?-yava? evd?n �?x?b i??-g�c? ged?nl?rini seyr etm?y? ba?lad?m, art?q g?tirdiyim sovqatlar? g�t�r�b d�?m?yin vaxt? oldu?unu anlad?m. Qay?d?b, yenid?n nar v? konyak qutular?n? g�t�rd�m v? sakitc? evin dalan?na endim. Ev sahibl?ri h?l? ?irin yuxuda idi. Taksi dayanaca?? el? d? uzaq deyildi, hardasa, yuz, y�z ?lli metr olard?, ancaq ?limd?ki y�k d? y�ng�l deyildi, bu m?saf? ���n a??r oldu?unu hiss etdim. Ba?qa c�r yolu da yox idi. Burda taksi dayanaca??nda ad?t?n taksil?r durmurdu, bo? g?lib dayana bil?rdi v? ?g?r s?rni?in yoxsa, �t�b ke�m?liydi. O q?d?r d? vaxt ke�m?di g�zl?m?y?, durdu?um s?kiy? bir bo? taksi yan ald? v? tez-t?l?sik minib �nvan? bildirdim. M?saf? o q?d?r d? uzaq deyildi, bilirdim ki, 10-15 d?qiq?y? orda
 yam. \n M?n taksid?n d�??nd? art?q 20 d?qiq? olard? ki, i? ba?lam??d?. Daxili telefona ?l atd?m, Valeri Yuryevi�i soru?dum, el? telefonu g�t�r?n �z�ym�?, �z�m� t?qdim etdim, tan?d?. Ondan a?a?? d�?m?sini xahi? etdim, art?q o, yan?mda idi. V? m�?yy?n fikir m�badil?sind?n sonra m�?yy?n sayda ka??z torbalar g?tirdil?r, nar v? konyaklar? ayr?-ayr? torbalara yerl??dirib, s?kkizinci m?rt?b?y? qalxd?q v? h?min h?diyy?l?ri onun ota??na qoyduq.\n Birinci g�r�??c?yim adam t?yin oldu?um ?�b?nin k�rasiyas?n? h?yata ke�ir?n m�avinl?rd?n biri olmal?yd?. Odur ki, h?diyy? torbalar?ndan birini g�t�r�b onun ota??na yax?nla?d?m. Katib?y? kimliyimi t?qdim etdim, o da daxili telefonla ona x?b?r verdi v? i�?ri girm?yimi bildirdi. ?limd? ki, h?diyy? torbas? il? birg? i�?ri daxil oldum, baxd?m ki, torban? qoyma?a el? bir yan-y�r? yoxdu, he� otaqda mebeld?sti d? yox idi ki, bu torban? ora yerl??dirim. S?dr m�avini bunu g�zl?mirmi? kimi, m?n? baxa-bax
 a:\n -Bu n?di bel?, -dey? maraqland?.\n -bu, qafqazl?lara m?xsus h?diyy?di, -dedim.\n - apar?n q?zlara paylay?n,-dedi.\n -m?n art?q onlara vermi??m,-dey?r?k, ?limd? ki, dolu ka??z torban? bir t?h?r divar?n dibin? qoydum.\n -yax??, yax?? oturun, -dey? m?n? yer g�st?rdi.\n M?n oturdum, o, daxili d�ym?ni ??had?t barma?? il? basd?, saniy? ke�m?mi? katib? q?z i�?ri daxil oldu.\n O, y�ks?k tonla, - o torban? g�t�r�n yan?n?zda saxlay?n,- dey?r?k q?za i?ar? etdi.\n S?dr m�avini �evik v? c?ld adam idi. Hiss etm?k olurdu ki, bank sisteminin ?hali v? t??kilatlarla ba?l? hiss?sini d?rind?n bilir. H?l? xarici d�vl?tl?rd? olan texnoloji prosesl?rd?n s�hb?t getm?s? d?, daxili v? o d�vr ���n ssri ?razisi ���n yeni v? m�asir hesab edil?n primitiv texnoloji prosesl?rin h?yata ke�irilm?sind?n s�hb?t ged? bil?rdi. O, m?niml? Az?rbaycan Respublikas?n?n �mumi iqtisadiyyat?ndan s�hb?t apararaq, �mumi anlay??? ?ld? etm?y? �al??d?. M?nd? yaranan t??ss�ra
 tdan hiss etdim ki,   art?q onun raz?l???n? ?ld? etmi??m v? bu t?sdiq m?s?l?si birinci m�avinl?  g�r�??nd?n sonra �z h?llini tapacaqd?r. I�imd? top el?diyim bu hissl?rl?, onun, daxili telefonu g�t�r�b, ad? v? familyas? m?n? b?lli olan birinci m�avinl?, m?nd?n raz? halda dan??mas? �st-�st? d�?d�. Sonra o, onun yan?nda adam olub-olmamas?n? d?qiql??dirib, q?buluna g?lm?ymizi m�?yy?nl??dir?r?k, -getdik, -dey?, m?n? i?ar? etdi. \n Art?q s?drin birinci m�avinin yan?nda idik. Birinci m�avin �z �?kisin? uy?un uca boy, �ox da k�k olmayan, haz?rl?ql?, intellektual bir adam t?siri ba???lad? m?n?. El? bizi d? qar??lama??ndan hiss etdim ki, bu adamla, i? bar?sind? hans? istiqam?td? ist?s?n s�hb?t aparmaq olar. �z�m� daxil?n s?rb?st hiss etdim v? daxili narahatl???ma he� bir ?sas olmad???n? q?t etdim.\n O, s?liq?-s?hmanla v? qay??ke?likl? ba??n? qald?r?b, ?vv?lc? bizim respublikam?z?n �mumi iqtisadi v?ziyy?tiyl? maraqland?, H?s?n Qas?movi�i so
 ru?du v? onun yax?? bir s?dr olaraq, i?k�zarl???n? v?  f?aliyy?tini qiym?tl?ndirdi.\nHiss etdim ki, h?r n?sn?ni i?l? d? �l�m?k olmaz v? bu adam?n i?d?n-g�cd?n ?lav?, h?m d? �mumu b???ri keyfiyy?tl?ri d? az deyil.\n O, daxili v?ziyy?tl? yana?? avropan?n v? d�nyan?n ?n m�t?r?qqi iqtisadi yenilikl?ri �z?rind?  dayand?. Bir �ox �lk?l?rin iqtisadi r?q?ml?rind?n, texnoloji prosesl?rind?n misallar �?kdi. Ancaq onda bunlar nisbi inki?a g�st?ricil?riydi, bu g�st?ricil?rin alt?nda xaricd? hans? texnoloji prosesl?rin durdu?u n? onun ���n, n? d? m?nim ���n anla??lan deyildi. T?kc? biz onu bilirdi ki, ?g?r Amerikada texnoloji prosesin bu v? ya dig?r sah?si 80-90 faiz t??kil edirdi s? bu ssrid? 10-20 faz �evr?sind? ola bil?rdi v? bu m�qayis?l?r bu istiqam?td? �z �l��s�n� v? geriliyini m�?yy?n ed? bil?rdi.\n B�t�n bunlara baxmayaraq, m?n ona diqq?tl? qulaq as?rd?m, ��nki bunlar m?nim ���n maraql? v? �n?mli idi. Bu �n?mli fak
 torlar?n  arxas?nda m?nim yeni i? f?aliyy?tim dururdu  v? bu i?l?ri yeni �sulda h?yata ke�irm?k ���n  �?kil?n v? g�st?ril?n misallar?n t?cr�b?d? istifad? imkanlar? m�tl?q v? laz?ml? idi, fikirl??dim.  H?r halda birinci m�avinin m?niml?, o d�vr�n diliyl? des?k, apard??? s�hb?tl?r, ataca??m add?mlar?n b�n�vr?si ���n  inaml? bir t?kan hesab edil? bil?rdi.\n Onun raz?l???n? v? xeyir-duas?n? ald?dan sonra, m?nd? yaranm?? xo? ?hval-ruhiyy? il? yan?ndan �?xd?m, n?z?rd? tutulmu? idar? v? ?�b? r?isl?riyl? g�r�?d�m, s�hb?t etdim, laz?m? tap??r?qlar? ald?m v? laz?m? s?n?dl?r? d?rk?narlar? qoydurdum. Art?q yeni v?zif?y? t?yinat?m bar?d? ?mrim haz?r idi, birc? qal?rd?  geriy? d�n�b, h?v?sl? i?? ba?lama??m.\n I?in sonuna lap az qalm??d?, fikirl??dim ki, Valeri Yuryevi�l? g�r�?m?liy?m, o da xo?b?xtlikd?n ba?qa bir i?in dal?nca getm?mi?di, yenid?n s?kkizinci m?rt?b?y? qalxd?m. Valeri Yuryevi� balacaboy, i? t?cr�b?sin? malik v? qay??ke? b
 ir adam idi, h?m d? m?nim t?yin oldu?um ?�b?nin, ssri �zr? sah? r?isi idi. H?mya??d olard?q. Ota??nda t?k idi. Ota??na daxil olub sevincimi onunla b�l�?d�rd�m, bir xeyli s�hb?t etdik, i?in maraql? olaca??n? bildirdi. M?nim beynimd? art?q i?d?n ?s?r ?lam?t qalmam??d?, birc? o qalm??d? ki, i?in sonu n? vaxt g?lib �atacaq v? bu sevinci, Valeri il? Moskvan?n M.Qorki k��?sind? yerl???n Bak? restoran?nda bir daha ??r?blana-??rablana b�l�?d�r?�?yik. Bel? anlarda, t?rs kimi  be?-on d?qiq?nin g?lib getm?si d? adama ���n dar?xd?r?c? olurmu?. Yax?? ki, s�hb?t maraql? al?nd? v? vaxt g?lib ke�di, s�hb?tin arxas?n? Bak? restoran?nda davam etdirm?yi q?rara ald?m. Onun m?nim niyy?timd?n x?b?ri yox idi. O, ona g?tirdiyim h?diyy?ni g�t�rm?k ist?y?nd?, bildirdim ki, onlar? mebel d?stinin g�z�nd? yerl??dirsin v? m?niml? getm?sini xahi? etdim. O, ?vv?l t?r?dd�t ets? d? sonra m?niml? raz?la?m?? formada otaqdan birg? �?xd?q. Bak? restoran? el? d? uzaq de
 yildi, i? yerind?n  g?z?-g?z? gets?k uzaqba?? on d?q?q?y? orda ola bil?rdik. Ancaq buna baxmayaraq, taksi yaxalad?m v? tez d? mindik, t?xmin?n iki d?qiq?d?n sonra restoran?n yan?nda idik. Restoran?n qap?s?nda duran ??vetsar?  yerl?rin tutldu?unu v? g�zl?m?ymizi bildirdi, y?ni yax?n d?qiq?l?rd? stolun birinin bo?alaca??n? bildik. Restoran?n gir?c?yind? var-g?l etdik, zaman da zaman idi, laz?m olan anlarda ir?lil?m?k bilmir ki, bilmir. N?hy?t, ??vetsar??n g�st?ri?in? ?sas?n bo? olan stola yax?nla?d?q, ?trafda qonaqlar �ox idi, ?sas?n d? az?rbaycanl?lar idi. G�z g?zdirs?m d? tan?? adam g�z�m? d?ym?di. \n Sifari?l?rimiz, m�vafiq raz?la?madan sonra Bak? m?tb?xin? aid kabablar v? ba?qa t?amlar oldu. Sa?l?qlar qar??l?ql? olsa da,  m?nim yeni v?zif? t?yinat?m ���n Valeri ?hat?li v? geni? nitq  etdi. Onun nitqind? ki, s?mimiyy?tin qanadlar? �iynimd?n as?la qalm??d?, ��nki sabahlar? i?in ir?lil?yi?i il? ba?l? hesabatlar? onun ?�b?sin? t?qdim ed?ciyimi bilirdim. Odu
 r ki, k�n�ls�z d? olsa, �z d?rin minn?tdarl???m? bildirdim. Art?q durma??n vaxt?yd?, vaxt gec?ni ?ymi?di, restoranda t?k?m-seyr?k adam qalm??d?, onlar da hallanan adamlara ox?ay?rd?. Stola xidm?t ed?n qulluq�u, restoran?n ba?lan?lmas?na be? d?qiq? qald???n? bildirdi. Stolun haqq-heab?n? yekunla?d?r?b restorandan �?xd?q, t?k?m-seyr?k taksil?r g�z? d?yirdi, el? bil t?l?sidiymizi bilirdi, biri saxlad?. �yr?ndim ki, Valeri d? m?n getdiyim istiqam?td? ya?ay?r. M?n Dimitrovski ?ossesinin ax?rna �atmam?? d�?m?li idim, o is? ax?r?nda.  N?zak?t xatirin? onu evin? kimi �t�r?si oldum v? h?min taksiyl? d?, yenid?n �z qald???m �nvana g?ldim.\n Qald???m �nvan yad deyildi, n? d? kiray? deyildi, uzun m�dd?t Moskvan?n sakini olmu? b�y�k qarda??m?m eviydi. Qarda??m da m?ni g�zl?diyind?n h?l? yatmam??d?. Yeni v?zif?mi   bir d? ??rablan?b t?krarlad?q. T?brik v? sa?l?qlardan sanra yetmi?inci ill?rd? burda bir pay?z vaxt? istirah?t ed?rk?n yazd???m ?eir yad?ma d�?d
 �. Ba?lad?m onu q?r?q-q?r?q yad?ma salma?a v? �z i�imd? p?�?ldama?a:\n Moskva...\n Bo? otaq...\n T?k?m, t?nhayam,\n Rusiyan?n bom-boz pay?z? kimi,\n Radio dal?as?nda ?Bak?? g?zm?kd?n\n Dar?xd?, titr?di q?lbimin simi.\n\n Moskva...\n Bo? otaq...\n Dinir ?T�rkiy??,\n Nazimin ?eiri, ??rqisi bu da,\n N? yaman uyudum bura harad?r,\n A? sa�l? Qorqudu g?tirdi yada.\n\n Moskva...\n Bo? otaq...\n �r?yimd? d?,\n Tar?ma �?kilmi? t?qvim ill?ri.\n Eh, bax, bu ?eir d? s?n? �atmad?,\n Soldu misralar?, soldu g�ll?ri.\n\n Iqlim d?yi?s? d? t?bi?timd?,\n Dilimi, s�z�m� yen? bilmir?m.\n Dem? ?eirliyim bura kimiymi?,\n Qagqazdan aral? din? bilmir?m.\n \n 1976-c? ild? Moskvada, bax, el? bu evd? yazd???m bu ?eirin yad?mda s?ralanm?? bu b?ndl?rini bir t?h?r v? bir daha,  dilim topuq vura-vura qarda??m ���n d? oxudum.  Gec?ni sevinc?k ke�irdik. Dan �z�n? az qalm??d?. Fikirl??dim ki, g�z�m�n hovurnu al?m, u�aq aya??na gecikm?yim, h?m d? bilet  m?s?l?si d? var idi
 . Oydu ki, yalandan da olsa divana uzand?m, ??rab?n meyxo?lu?undan yuxuya gets?m d?, stol�st� saat?n z?ngin? oyand?m. Saat  on r?q?minin d?qiq?l?rini yenic? add?mlama?a ba?lam??d?, h?l? ki, on be?inci add?m?ndayd?. �z�m� vanna ota??na sald?m, �z�m?, g�z�m? isti-soyuq qar???q su vurdum, d?smallan?b, vanna ota??ndan �?xd?m.  Qarda??m? oyatma?a ?lim g?lm?s? d?, m?cburuydum oyatma?a, ��nki qap?n?n a�?lmas? v? ba?lan?lmas? var?yd?. O, bunu hiss etmi? kimi, getm?yimi soru?du, bir st?kan p�rr r?ngi �ay i�ib, bozarm?? hvada taksi dayanaca??na t?l?sdim v? bir taksiy? oturub hava liman?na y�n ald?m.\n Hava liman?n?n bilet kassas?nda n�vb? �ox idi, dayand?m, n?h?y?t ki, n�vb?m �atd?. Moskvadan Bak?ya u�an u�a??n n�vb?ti reysin? bilet ald?m. H?l? vaxta var idi. Yenid?n ax?amk? mar?urutu bir d? t?krar ed?si oldum. M.Qorki k��?siyl? �z� yuxar? yolland?m. Qar??mda yen? ?Bak??restoran?yd?. G�nd�z oldu?undan restoranda adam az idi. Bo? stol
 lar?n birin? yax?nla??b ( onsuz da stollar  bo? idi),\nOturdum v? menyunu g�zd?n ke�irm?y? ba?lad?m.  Az?rbaycan m?tb?xin? aid h?r n?sn? il? z?ngin idi  restoran. \n Ba??m? qald?rd?m ki, qar??mda bir adam durub, ancaq tan?mad?m. Ad?m? �?k?r?k, -salam, tan?mad?n m?ni?, -dey?r?k, m?ni s�zd�.\n -tan?? g?lirsiniz, ancaq xat?rlaya bilmir?m, ad?n?z da yad?mdan �?x?b, -dedim.\n -bir yax?? fikirl?? Asif ?f?ndiyevl? s?nd?n ba?qa kiml?r yax?nl?q edirdi.\n - h?, h?, -dedim,-yad?ma d�?d�, siz K�rd?mird?n idiniz, ad?n?z da Z?man?t, famiyan?z da B?kirov.  Siz Moskvaya aspiranturaya g?ldiniz, s?hv etmir?ms?, m�vzunuz da ?Heygel f?ls?f?sind? ??rq m�vzusu?  idi. �z� d? s?hv etmir?ms?, �. Hac?b?yov ad?na Bak? D�vl?t Konservatoriyas?n?n ?ahnaz adl? bir m�?llim?si var idi siz onunla birg? g?ldiniz. O vaxtlar deyirdiniz ki, sizi �� f?lak?t g�zl?yir, bax, indi g�r�r?m ki, he� bir f?lak?tiniz filan yoxdu. Y?qin ki, m�dafi?nizi d? etmisiniz.\n O, d?rind?n k�k
 s �t�rd�, g�l�ms�nd� v? soru?du, - de g�r�m yem?k sifari?i vermis?n, ya yox, s�hb?timizi rahat ed?k, -dedi.\n Stola qulluq ed?n a? xalatl? q?z, sanki bizi g�rm�rd�, b?lk? d? s�hb?t etdiyimizi g�r�b rahat buraxm??d? ki, yan?mdak? adam da otursun v? sonra biz? yax?nla?s?n. \n N?hay?t ki, stola qulluq ed?n a? xalatl? q?z biz? yax?nla?d?. Art?q m?n? ayd?n idi ki, m?nim bu k�hn? dostum da restorana indi daxil olub v? �mumi bir sifari? ver?si olduq. Ancaq u�aq aya??nda olsam da k�hn? dostun x?tirn? d?y? bilm?dim, G?nc? konyak?n? t?rifl?di v? s�hb?timizi d? G?nc? konyak? alt?nda ba?lad?q.\n O, �z�n� m?n? tutub, - h?, o dediyin f?lak?tin ikisi oldu da, qurtard? da. ���nc� f?lak?ti g�zl?yir?m, n?d?n ibar?t olaca??n? a�?qlaya bilm?y?c?y?m, g?r?k �z�rl� hesab ed?siniz, ancaq vaxt? g?l?nd? dey?r?m. O dediyin m�vzunu m�dafi? etmi??m, hal-haz?rda doktorluk dissertasiyas? �z?rind? �al???ram, ancaq d?y?s?n �m�r buna imkan
  verm?y?c?k. ?ahnaz da ?milli m�siqimizin f?ls?f?sind?n? m�dafi? etdi v? hal-haz?rda   Bak? D�vl?t Konservatoriyas?nda m�?llim i?l?yir. H?l? ki, Bak?ya gec-gec gedir?m. A. ?f?ndiyevi g�rm�rs?n ki, onun da taleyi d�z g?tirm?di.  Az?rbaycanda A. ?f?ndiyev �ox m�tail?li bir filosofdu, m?n f?ls?f?nin ?n d?rin qatlar?n? ondan �yr?nmi??m. El? Heygel f?ls?f?sind? ??rq m�vzusunu da o, m?nim a?l?ma salm??d?. Ancaq A. ?f?ndiyevin bir �at??mayan c?h?ti oldu ki, f?ls?f?ni sisteml??dir? bilm?di, bu da onun t?k?bb�r� il? ba?l? bir m?s?l? idi. M?n h?mi?? onunla s�hb?tl?rimd? bu m?s?l?l?ri ona xat?rlad?rd?m v? xahi? edirdim ki, b�t�n bu fikirl?ri bir m?craya g?tirsin, f?ls?f?ni yeni istiqam?td? �mumil??dirsin v? �mumi bir n?z?riyy? hal?na sals?n. O, m?niml? raz?la?m??d?, ancaq �z�n� toparlama?a vaxt? yox idi, ��nki h?mi?? g?ncl?r aras?nda ?oca??? m�q?dd?sl??dir?r?k, bu oca??n ?�l?sini h?r ?eyd?n �st�n tuturdu. O, f?ls?f?ni V?t?n sevgisind?, oc
 aq sevgisind?, b�y�k t�rkl�y�n yenilm?zliyind?, ad?t?t ?n?n?sind? g�r�rd� v? vaxt?n? bu g�r�?l?r? h?sr etdiyind?n b?zi m?s?l?l?r bitm?mi? qald?, buna heyifsl?nm?kd?n ba?qa �ar?miz yoxdu. \n M?n maraql? m�vzuya m�daxil? ed?r?k, - Asif ?f?ndiyevd? G�y Tanr? inanc? var idi, o, yerin d?, g�y�n d?, canl? v? cans?z?n da ilahi bir  q�vv? t?rfind?n yarad?ld???n?n izah etdiyinin sizinl? birg? ?ahidiyik. Bir t�rk olaraq saz?m?z? da b�t�n t�rk d�nyas?n?n bayra?? elan etmi?di, qalan al?tl?ri is? g?lm? hesab edirdi.Az?rbaycan D�vl?t Universitetind? ke�irdiyi b?dii-f?ls?fi gec?l?r?, saz-s�z ustas?, ?mran H?s?nov, ?kb?r C?f?rov, ?dal?t N?sibov ?Misrisi?yl?, ?C?ngi?siyl?, ?Dilq?mi?siyl?, ?Ruhani?siyl?, ?Q?hraman??s?yla, Az?rbaycan?n g�rk?mli xan?nd?l?rind?n Yaqub M?mm?dov, Can?li ?kb?rov ?M?nsuriyy??siyl?, ??anaz??yla, Kamil C?lilov q?boyundak? inc? ritml?riyl? qat?lard?. �mumiyy?tl? o, b�t�n f?ls?f?nin d�y�n n�qt?sini T�rk D�nyas?
  adl? b�y�k bir co?rafiyan?n m�siqisind?, bayat?s?nda, a??s?nda, ac?s?nda g�r?rdi. Ancaq m?n d? sizinl? raz?yam ki, o, f?ls?f?ni bu co?rafi ?razi �zr? sisteml??dir? bilm?di, bu da ssri kimi bir imperiyan?n ideologiyas? il? m�bariz?y? s?rf olunan zaman itgisiyl? ba?l?yd?. Tanr? ona m�st?qilliyimizd?n sonra laz?m? �m�r v? ya?am ���n m�vafiq t?minat vers?ydi, y?qin ki, o, f?ls?f?nin m�?yy?n sah?l?rini �mumil??dir? bil?c?kdi.\n S�hb?t? ba??m?z qar??d???ndan yem?kl?rin stola d�z�ld�y�nd?n x?b?rimiz yox idi. G�z�m  ya??la �alan ?�?? butulkada g?tirilmi? G?nc? konyak?na sata?anda Z?man?tin �z�n? baxd?m. O, m?ni hiss edirmi?  kimi, ?�?? butulkaya ?l atd?, ilk bad?l?ri t?sad�fi g�r�?m�z�n daimi olmas? ??r?fin? nu? etdik. N�vb?ti  bad?l?rin d? s?b?bkarlar? tap?ld?  v? nu? edib qalxd?q. Hesab? is? Z?man?t ver?si oldu. O, bunu onunla izah etdi ki, o m?nd?n b�y�kd� v? m?n Moskvada qona?am, o da ba?a d�?�l?n bir n?sn? idi
 . \n O, m?nim u�u? vaxt?ma az qald???n? hiss edib, m?ni �t�rm?yi, yola salma??  q?rara ald?, m?n buna raz? olmad?m. O, qara kiprikl?rinin v? qara b??lar?n?n alt?ndan b?rq vuran t?b?ss�ml� g�l�?l?riyl?, m?ni, el? restoran?n h?y?tind?n yola sald? v? x�dahifizl??ib ayr?ld?q.\n Bir ba?a hava liman?na g?ldim, u�u?un vaxt?na h?l? bir saat qalm??d?."}},
+{"photo":{"from":{"name":"Thami de Oliveira","id":"100003470164026"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yz/r/StEh3RhPvjk.gif","id":"100003470164026_390971624361840","created_time":"2013-08-28T00:26:45+0000","type":"photo","updated_time":"2013-08-28T00:30:15+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=390971007695235&set=pcb.390971624361840&type=1&relevant_count=3","object_id":"390971007695235","message":"MENINAS QUE TIVEREM INTERESSE NESSE VESTIDO TEM NA COR BRANCO, AMARELO, AZUL BIC, VERDE E PRETO, COMENTA COM A COR QUE FOR QUERER PRA EU PODER FAZER A ENCOMENDA, S� VOU TRAZER DE QUEM COMENTAR !!!","picture":"http://photos-h.ak.fbcdn.net/hphotos-ak-prn1/1236641_390971007695235_471308162_t.jpg"}},
+{"photo":{"name":"Timeline Photos","story":"Luis Carlos shared Minha V� Ta Maluca's photo.","from":{"name":"Luis Carlos","id":"100003127462424"},"application":{"name":"Photos","id":"2305272732"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"0":[{"name":"Luis Carlos","id":"100003127462424","type":"user","length":11,"offset":0}],"19":[{"name":"Minha V� Ta Maluca","id":"282072935240952","type":"page","length":18,"offset":19}]},"id":"100003127462424_439260802854831","created_time":"2013-08-28T00:26:05+0000","type":"photo","caption":"Recomendo : Sem Sono\r\nRecomendo : Eu Amo Viajar","updated_time":"2013-08-28T00:26:05+0000","privacy":{"value":""},"link":"http://www.facebook.com/photo.php?fbid=468887093226201&set=a.445796168868627.1073741872.282072935240952&type=1","object_id":"468887093226201","message":". . . . ja, fui bic�o em varias, e ainda chegava comprimentando todo mundo . . . . ","properties":[{"href":"http://www.facebook.com/votamaluc
 a?ref=stream","name":"By","text":"Minha V� Ta Maluca"}],"picture":"http://photos-a.ak.fbcdn.net/hphotos-ak-ash3/1256526_468887093226201_1763899420_s.jpg"}},
+{"photo":{"is_hidden":true,"name":"Timeline Photos","story":"Giovanna Lopes shared 4:20's photo.","from":{"name":"Giovanna Lopes","id":"100003553147558"},"application":{"name":"Links","id":"2309869772"},"icon":"http://static.ak.fbcdn.net/rsrc.php/v2/yD/r/aS8ecmYRys0.gif","story_tags":{"22":[{"name":"4:20","id":"304764186208953","type":"page","length":4,"offset":22}],"0":[{"name":"Giovanna

<TRUNCATED>

[11/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-gnip/gnip-edc-flickr/src/test/resources/FlickrEDC.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-gnip/gnip-edc-flickr/src/test/resources/FlickrEDC.xml b/streams-contrib/streams-provider-gnip/gnip-edc-flickr/src/test/resources/FlickrEDC.xml
index 667d9b1..8774613 100644
--- a/streams-contrib/streams-provider-gnip/gnip-edc-flickr/src/test/resources/FlickrEDC.xml
+++ b/streams-contrib/streams-provider-gnip/gnip-edc-flickr/src/test/resources/FlickrEDC.xml
@@ -1,64 +1,64 @@
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564697996</id>  <published>2013-08-21T15:42:53Z</published>  <updated>2013-08-21T15:42:53Z</updated>  <title type="html">Mygor posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/89791507@N04/9564697996"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per_
 page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="ht
 tp://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564697996</id>    <title type="text">Ensimismada B&amp;N</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/89791507@N04/9564697996"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7320/9564697996_7f70d41157.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7320/9564697996_7f70d41157_t.jpg"/>  </activity:object>  <author>    <name>Mygor</name>    <uri>http://www.flickr.com/people/89791507@N04/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr
 .com/people/89791507@N04"/>    <id>http://www.flickr.com/people/89791507@N04</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>2596523316</id>  <published>2013-08-21T15:48:00Z</published>  <updated>2013-08-21T15:48:00Z</updated>  <title type="html">Snuffy posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/81418524@N00/2596523316"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per
 _page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="h
 ttp://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:2596523316</id>    <title type="text">Notre Dame Cathedral, Lyon, France</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/81418524@N00/2596523316"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3281/2596523316_5189fc2849.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3281/2596523316_5189fc2849_t.jpg"/>    <category term="france"/>    <category term="beautiful"/>    <category term="lyon"/>    <category term="notredamecathedral"/>    <category term="worldtrekker"/>    <category term="artofimages"/>    <georss:point xmlns:georss="http://www.georss.org/georss">45.762212 4.822815</georss:point>  </activity:object>  <author>    <name>Snuffy<
 /name>    <uri>http://www.flickr.com/people/81418524@N00/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/81418524@N00"/>    <id>http://www.flickr.com/people/81418524@N00</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561899727</id>  <published>2013-08-21T15:41:41Z</published>  <updated>2013-08-21T15:41:41Z</updated>  <title type="html">Chrisser posted an image to Flickr</title>  <summary type="text">Light two-tone pink rose mallow photo made with the flower converted to black and white and the background converted to sepia using Photoshop CS5.1</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/51405405@N00/9561899727"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%
 22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystre
 a.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561899727</id>    <title type="text">Light Two-Tone Pink Rose Mallow Selective Colour B&amp;W Over Sepia 005</title>    <content type="html">Light two-tone pink rose mallow photo made with the flower converted to black and white and the background converted to sepia using Photoshop CS5.1</content>    <link rel="alternate" href="http://www.flickr.com/photos/51405405@N00/9561899727"/>    <link rel="enclosure" href="http://farm6.static.flickr.com/5500/9561899727_5c0299247f.jpg"/>    <link rel="preview" href="http://farm6.static.flickr.com/5500/9561899727_5c0299247f_t.jpg"/>    <ca
 tegory term="flowers"/>    <category term="summer"/>    <category term="ontario"/>    <category term="canada"/>    <category term="nature"/>    <category term="photoshop"/>    <category term="garden"/>    <category term="gardening"/>    <category term="hibiscus"/>    <category term="fourseasons"/>    <category term="malvaceae"/>    <category term="closeups"/>    <category term="selectivecolour"/>    <category term="rosemallows"/>    <category term="canonefs1855mmf3556islens"/>    <category term="canoneosrebelt1i"/>  </activity:object>  <author>    <name>Chrisser</name>    <uri>http://www.flickr.com/people/51405405@N00/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/51405405@N00"/>    <id>http://www.flickr.com/people/51405405@N00</id>  </activity:actor>  <gnip:matching_r
 ules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561813803</id>  <published>2013-08-21T15:44:21Z</published>  <updated>2013-08-21T15:44:21Z</updated>  <title type="html">joyfullytaken posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/98770454@N05/9561813803"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&
 amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:acti
 vity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561813803</id>    <title type="text">focus b&amp;w</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/98770454@N05/9561813803"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7402/9561813803_e88531c60d.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7402/9561813803_e88531c60d_t.jpg"/>  </activity:object>  <author>    <name>joyfullytaken</name>    <uri>http://www.flickr.com/people/98770454@N05/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://
 www.flickr.com/people/98770454@N05"/>    <id>http://www.flickr.com/people/98770454@N05</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561937071</id>  <published>2013-08-21T15:47:45Z</published>  <updated>2013-08-21T15:47:45Z</updated>  <title type="html">Graham Knott posted an image to Flickr</title>  <summary type="text">At The Ecclesbourne Valley Railway, Wirksworth&lt;b&gt;Click here to show my photostream &lt;a href="http://www.flickr.com/photos/grahamknott/?details=1"&gt;not justified&lt;/a&gt;&lt;/b&gt;</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/8118124@N07/9561937071"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20spark
 le%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <servi
 ce:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561937071</id>    <title type="text">Lancashire and Yorkshire Railway class 27</title>    <content type="html">At The Ecclesbourne Valley Railway, Wirksworth&lt;b&gt;Click here to show my photostream &lt;a href="http://www.flickr.com/photos/grahamknott/?details=1"&gt;not justified&lt;/a&gt;&lt;/b&gt;</content>    <link rel="alternate" href="http://www.flickr.com/photos/8118124@N07/9561937071"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3733/9561937071_4608e46abf.jpg"/>    <link rel="preview" href="http://farm4.st
 atic.flickr.com/3733/9561937071_4608e46abf_t.jpg"/>    <category term="blackandwhite"/>    <category term="derbyshire"/>    <category term="wirksworth"/>    <category term="ecclesbournevalleyrailway"/>    <category term="lancashireandyorkshirerailwayclass27"/>    <georss:point xmlns:georss="http://www.georss.org/georss">53.083108 -1.568813</georss:point>  </activity:object>  <author>    <name>Graham Knott</name>    <uri>http://www.flickr.com/people/8118124@N07/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/8118124@N07"/>    <id>http://www.flickr.com/people/8118124@N07</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "p
 aper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564608636</id>  <published>2013-08-21T15:44:16Z</published>  <updated>2013-08-21T15:44:16Z</updated>  <title type="html">joyfullytaken posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/98770454@N05/9564608636"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&
 amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:acti
 vity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564608636</id>    <title type="text">like mother like son in b&amp;w</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/98770454@N05/9564608636"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3726/9564608636_c719bc8bde.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3726/9564608636_c719bc8bde_t.jpg"/>  </activity:object>  <author>    <name>joyfullytaken</name>    <uri>http://www.flickr.com/people/98770454@N05/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length
 ="0" href="http://www.flickr.com/people/98770454@N05"/>    <id>http://www.flickr.com/people/98770454@N05</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564718050</id>  <published>2013-08-21T15:47:17Z</published>  <updated>2013-08-21T15:47:17Z</updated>  <title type="html">Ngoc T posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/39582444@N03/9564718050"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per
 _page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="h
 ttp://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564718050</id>    <title type="text">IMG_2943</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/39582444@N03/9564718050"/>    <link rel="enclosure" href="http://farm6.static.flickr.com/5511/9564718050_ed0ae1c669.jpg"/>    <link rel="preview" href="http://farm6.static.flickr.com/5511/9564718050_ed0ae1c669_t.jpg"/>    <category term="street"/>    <category term="city"/>    <category term="boy"/>    <category term="bw"/>    <category term="cute"/>    <category term="girl"/>    <category term="smile"/>    <category term="canon"/>    <category term="kid"/>    <category term="vietnam"/>    <category term="study"/>    <category term="chi"/>    <category term="ho"/>    <category 
 term="minh"/>    <category term="60d"/>  </activity:object>  <author>    <name>Ngoc T</name>    <uri>http://www.flickr.com/people/39582444@N03/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/39582444@N03"/>    <id>http://www.flickr.com/people/39582444@N03</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564711068</id>  <published>2013-08-21T15:45:11Z</published>  <updated>2013-08-21T15:45:11Z</updated>  <title type="html">shahreen | amri posted an image to Flickr</title>  <summary type="text">Location :  &lt;b&gt;Detian Town, Detian, Nanning, Guangxi, China&lt;/b&gt;&lt;b&gt;Email : shahreenphoto@gmail.comWebsite : &lt;a href="http://www.shahreenphoto.com" rel="nofollow"&gt;www.shahreenphoto.com&lt;/a&gt;&lt;/b&gt;&lt;b&gt;Thanks for the visit, comments, awards, invitations and favorites.&lt;/b&gt;&lt;b&gt;Please don't use this image on websites, blogs or other mediawithout my explicit permission.&lt;/b&gt;&lt;b&gt;&#xA9; All rights reserved &lt;/b&gt;</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/31827944@N04/9564711068"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?metho
 d=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("s
 parkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564711068</id>    <title type="text">Down Stream of Detian</title>    <content type="html">Location :  &lt;b&gt;Detian Town, Detian, Nanning, Guangxi, China&lt;/b&gt;&lt;b&gt;Email : shahreenphoto@gmail.comWebsite : &lt;a href="http://www.shahreenphoto.com
 " rel="nofollow"&gt;www.shahreenphoto.com&lt;/a&gt;&lt;/b&gt;&lt;b&gt;Thanks for the visit, comments, awards, invitations and favorites.&lt;/b&gt;&lt;b&gt;Please don't use this image on websites, blogs or other mediawithout my explicit permission.&lt;/b&gt;&lt;b&gt;&#xA9; All rights reserved &lt;/b&gt;</content>    <link rel="alternate" href="http://www.flickr.com/photos/31827944@N04/9564711068"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7411/9564711068_6db051ba24.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7411/9564711068_6db051ba24_t.jpg"/>  </activity:object>  <author>    <name>shahreen | amri</name>    <uri>http://www.flickr.com/people/31827944@N04/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/31827944@N04"/>    <id>http
 ://www.flickr.com/people/31827944@N04</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564733642</id>  <published>2013-08-21T15:49:23Z</published>  <updated>2013-08-21T15:49:23Z</updated>  <title type="html">Fe_Lima posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/88158633@N04/9564733642"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;pe
 r_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="
 http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564733642</id>    <title type="text">Curves</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/88158633@N04/9564733642"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3830/9564733642_b222e1d378.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3830/9564733642_b222e1d378_t.jpg"/>    <category term="bw"/>    <category term="contrast"/>    <category term="nikon"/>    <category term="curves"/>    <category term="pretoebranco"/>    <category term="d3100"/>  </activity:object>  <author>    <name>Fe_Lima</name>    <uri>http://www.flickr.com/people/88158633@N04/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">
     <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/88158633@N04"/>    <id>http://www.flickr.com/people/88158633@N04</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561912229</id>  <published>2013-08-21T15:43:41Z</published>  <updated>2013-08-21T15:43:41Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561912229"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561912229</id>    <title type="text">s_c_b (11 of 340)b</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561912229"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7301/9561912229_fa0d3377ec.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7301/9561912229_fa0d3377ec_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-t
 ype>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561896689</id>  <published>2013-08-21T15:42:41Z</published>  <updated>2013-08-21T15:42:41Z</updated>  <title type="html">andrea gaspare posted an image to Flickr</title>  <summary type="text">So many thoughts</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/97827409@N06/9561896689"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_
 tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activit
 y:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561896689</id>    <title type="text">Please, stop.</title>    <content type="html">So many thoughts</content>    <link rel="alternate" href="http://www.flickr.com/photos/97827409@N06/9561896689"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3707/9561896689_884db05efa.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3707/9561896689_884db05efa_t.jpg"/>    <category term="bw"/>    <category term="white"/>    <category term="black"/>    <category term="me"/>    <category term="please"/>    <category term="head"/>    <category term="boom"/>    <category term="help"/>    <category term="thoughts"/>    <category term="stop"/>    <category term="thinking"/>    <category te
 rm="confused"/>    <category term="sorrow"/>    <category term="exploding"/>  </activity:object>  <author>    <name>andrea gaspare</name>    <uri>http://www.flickr.com/people/97827409@N06/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/97827409@N06"/>    <id>http://www.flickr.com/people/97827409@N06</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561900893</id>  <published>2013-08-21T15:44:06Z</published>  <updated>2013-08-21T15:44:06Z</updated>  <title type="html">shinymorning posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/85563462@N03/9561900893"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&a
 mp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activ
 ity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561900893</id>    <title type="text">untitled</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/85563462@N03/9561900893"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7360/9561900893_a6984bf791.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7360/9561900893_a6984bf791_t.jpg"/>    <category term="portrait"/>    <category term="blackandwhite"/>    <category term="bw"/>    <category term="blackwhite"/>    <category term="childrenportrait"/>  </activity:object>  <author>    <name>shinymorning</name>    <uri>http://www.flickr.com/people/85563462@N03/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">
     <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/85563462@N03"/>    <id>http://www.flickr.com/people/85563462@N03</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564731176</id>  <published>2013-08-21T15:48:03Z</published>  <updated>2013-08-21T15:48:03Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564731176"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564731176</id>    <title type="text">s_c_b (15 of 340)</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564731176"/>    <link rel="enclosure" href="http://farm3.static.flickr.com/2834/9564731176_21764f4cbf.jpg"/>    <link rel="preview" href="http://farm3.static.flickr.com/2834/9564731176_21764f4cbf_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-ty
 pe>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561899795</id>  <published>2013-08-21T15:41:42Z</published>  <updated>2013-08-21T15:41:42Z</updated>  <title type="html">Chrisser posted an image to Flickr</title>  <summary type="text">Light two-tone pink rose mallow photo made with the flower converted to sepia and the background converted to black and white using Photoshop CS5.1</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/51405405@N00/9561899795"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%
 22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystre
 a.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561899795</id>    <title type="text">Light Two-Tone Pink Rose Mallow Selective Colour Sepia Over B&amp;W 005</title>    <content type="html">Light two-tone pink rose mallow photo made with the flower converted to sepia and the background converted to black and white using Photoshop CS5.1</content>    <link rel="alternate" href="http://www.flickr.com/photos/51405405@N00/9561899795"/>    <link rel="enclosure" href="http://farm6.static.flickr.com/5533/9561899795_f1fa0806a7.jpg"/>    <link rel="preview" href="http://farm6.static.flickr.com/5533/9561899795_f1fa0806a7_t.jpg"/>    <ca
 tegory term="flowers"/>    <category term="summer"/>    <category term="ontario"/>    <category term="canada"/>    <category term="nature"/>    <category term="photoshop"/>    <category term="garden"/>    <category term="gardening"/>    <category term="hibiscus"/>    <category term="fourseasons"/>    <category term="malvaceae"/>    <category term="closeups"/>    <category term="selectivecolour"/>    <category term="rosemallows"/>    <category term="canonefs1855mmf3556islens"/>    <category term="canoneosrebelt1i"/>  </activity:object>  <author>    <name>Chrisser</name>    <uri>http://www.flickr.com/people/51405405@N00/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/51405405@N00"/>    <id>http://www.flickr.com/people/51405405@N00</id>  </activity:actor>  <gnip:matching_r
 ules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561934003</id>  <published>2013-08-21T15:47:18Z</published>  <updated>2013-08-21T15:47:18Z</updated>  <title type="html">Ngoc T posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/39582444@N03/9561934003"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;per
 _page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity="h
 ttp://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561934003</id>    <title type="text">IMG_3109</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/39582444@N03/9561934003"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3682/9561934003_e86fb18c5f.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3682/9561934003_e86fb18c5f_t.jpg"/>    <category term="street"/>    <category term="city"/>    <category term="boy"/>    <category term="bw"/>    <category term="cute"/>    <category term="girl"/>    <category term="smile"/>    <category term="canon"/>    <category term="kid"/>    <category term="vietnam"/>    <category term="study"/>    <category term="chi"/>    <category term="ho"/>    <category 
 term="minh"/>    <category term="60d"/>  </activity:object>  <author>    <name>Ngoc T</name>    <uri>http://www.flickr.com/people/39582444@N03/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/39582444@N03"/>    <id>http://www.flickr.com/people/39582444@N03</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564727014</id>  <published>2013-08-21T15:47:23Z</published>  <updated>2013-08-21T15:47:23Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564727014"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564727014</id>    <title type="text">s_c_b (14 of 340)</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564727014"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3720/9564727014_629dc73bd8.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3720/9564727014_629dc73bd8_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-ty
 pe>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561849315</id>  <published>2013-08-21T15:40:47Z</published>  <updated>2013-08-21T15:40:47Z</updated>  <title type="html">Ceshe posted an image to Flickr</title>  <summary type="text">Carrer a la Festa Major de Gr&#xE0;cia 2013</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/20661392@N07/9561849315"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name
 ,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:p
 rovider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561849315</id>    <title type="text">letra B, letra B...</title>    <content type="html">Carrer a la Festa Major de Gr&#xE0;cia 2013</content>    <link rel="alternate" href="http://www.flickr.com/photos/20661392@N07/9561849315"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3706/9561849315_1e7c179470.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3706/9561849315_1e7c179470_t.jpg"/>    <category term="barcelona"/>    <category term="catalunya"/>    <category term="gr&#xE0;cia"/>    <category term="abecedario"/>    <category term="fmgr&#xE0;cia"/>    <category term="ceshe"/>    <georss:point xmlns:georss="http://www.georss.org/georss">41.398242 2.1
 57983</georss:point>  </activity:object>  <author>    <name>Ceshe</name>    <uri>http://www.flickr.com/people/20661392@N07/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/20661392@N07"/>    <id>http://www.flickr.com/people/20661392@N07</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561897101</id>  <published>2013-08-21T15:41:17Z</published>  <updated>2013-08-21T15:41:17Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561897101"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561897101</id>    <title type="text">s_c_b (10 of 340)</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561897101"/>    <link rel="enclosure" href="http://farm3.static.flickr.com/2805/9561897101_fcfa54273c.jpg"/>    <link rel="preview" href="http://farm3.static.flickr.com/2805/9561897101_fcfa54273c_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-ty
 pe>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561889245</id>  <published>2013-08-21T15:40:10Z</published>  <updated>2013-08-21T15:40:10Z</updated>  <title type="html">arbyreed posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/19779889@N00/9561889245"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,description&amp;p
 er_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:activity=
 "http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561889245</id>    <title type="text">Wooden</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/19779889@N00/9561889245"/>    <link rel="enclosure" href="http://farm4.static.flickr.com/3700/9561889245_5715ff3dd1.jpg"/>    <link rel="preview" href="http://farm4.static.flickr.com/3700/9561889245_5715ff3dd1_t.jpg"/>    <category term="wood"/>    <category term="blackandwhite"/>    <category term="bw"/>    <category term="closeup"/>    <category term="close"/>    <category term="arbyreed"/>  </activity:object>  <author>    <name>arbyreed</name>    <uri>http://www.flickr.com/people/19779889@N00/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0
 /">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/19779889@N00"/>    <id>http://www.flickr.com/people/19779889@N00</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564695430</id>  <published>2013-08-21T15:42:19Z</published>  <updated>2013-08-21T15:42:19Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564695430"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564695430</id>    <title type="text">s_c_b (11 of 340)a</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9564695430"/>    <link rel="enclosure" href="http://farm3.static.flickr.com/2826/9564695430_c0c6472824.jpg"/>    <link rel="preview" href="http://farm3.static.flickr.com/2826/9564695430_c0c6472824_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-t
 ype>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561886235</id>  <published>2013-08-21T15:40:05Z</published>  <updated>2013-08-21T15:40:05Z</updated>  <title type="html">dmtaylor14213 posted an image to Flickr</title>  <summary type="text">The Beaconsfield Inn, B &amp;amp; B, Victoria, B.C.</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/36740584@N05/9561886235"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_u
 pload,owner_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <ico
 n/>  </service:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561886235</id>    <title type="text">PacificNorthwestVictoria136</title>    <content type="html">The Beaconsfield Inn, B &amp;amp; B, Victoria, B.C.</content>    <link rel="alternate" href="http://www.flickr.com/photos/36740584@N05/9561886235"/>    <link rel="enclosure" href="http://farm6.static.flickr.com/5477/9561886235_dc71961017.jpg"/>    <link rel="preview" href="http://farm6.static.flickr.com/5477/9561886235_dc71961017_t.jpg"/>  </activity:object>  <author>    <name>dmtaylor14213</name>    <uri>http://www.flickr.com/people/36740584@N05/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activity
 strea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/36740584@N05"/>    <id>http://www.flickr.com/people/36740584@N05</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561949129</id>  <published>2013-08-21T15:49:31Z</published>  <updated>2013-08-21T15:49:31Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561949129"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </se
 rvice:provider>  <activity:verb xmlns:activity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9561949129</id>    <title type="text">s_c_b (16 of 340)</title>    <content type="html">wedding hartlepool hart village staincliffe hotel</content>    <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561949129"/>    <link rel="enclosure" href="http://farm3.static.flickr.com/2892/9561949129_a6216bfba1.jpg"/>    <link rel="preview" href="http://farm3.static.flickr.com/2892/9561949129_a6216bfba1_t.jpg"/>    <category term="sarahmarkweddingmay2013"/>  </activity:object>  <author>    <name>Torrit</name>    <uri>http://www.flickr.com/people/58789603@N02/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-ty
 pe>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/58789603@N02"/>    <id>http://www.flickr.com/people/58789603@N02</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9564685650</id>  <published>2013-08-21T15:41:38Z</published>  <updated>2013-08-21T15:41:38Z</updated>  <title type="html">shoopshoopshoop posted an image to Flickr</title>  <summary type="text"></summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/60228304@N06/9564685650"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,owner_name,tags,geo,machine_tags,descriptio
 n&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>    <gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:rule>  </source>  <service:provider xmlns:service="http://activitystrea.ms/service-provider">    <name>Flickr</name>    <uri>http://www.flickr.com/</uri>    <icon/>  </service:provider>  <activity:verb xmlns:ac
 tivity="http://activitystrea.ms/spec/1.0/">http://activitystrea.ms/schema/1.0/post</activity:verb>  <activity:object xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/photo</activity:object-type>    <id>object:9564685650</id>    <title type="text">150/365</title>    <content type="html"></content>    <link rel="alternate" href="http://www.flickr.com/photos/60228304@N06/9564685650"/>    <link rel="enclosure" href="http://farm8.static.flickr.com/7327/9564685650_f7cf3a3d19.jpg"/>    <link rel="preview" href="http://farm8.static.flickr.com/7327/9564685650_f7cf3a3d19_t.jpg"/>    <category term="blackandwhite"/>    <category term="bw"/>    <category term="water"/>    <category term="square"/>    <category term="bottle"/>    <category term="day150"/>    <category term="project365"/>    <category term="day150365"/>    <category term="may2013"/>    <category term="project3652013"/>    <category term="3652013"/>    <category term="
 365the2013edition"/>    <category term="projectlife365"/>    <category term="30may13"/>  </activity:object>  <author>    <name>shoopshoopshoop</name>    <uri>http://www.flickr.com/people/60228304@N06/</uri>  </author>  <activity:actor xmlns:activity="http://activitystrea.ms/spec/1.0/">    <activity:object-type>http://activitystrea.ms/schema/1.0/person</activity:object-type>    <link rel="alternate" type="text/html" length="0" href="http://www.flickr.com/people/60228304@N06"/>    <id>http://www.flickr.com/people/60228304@N06</id>  </activity:actor>  <gnip:matching_rules>    <gnip:matching_rule rel="source" tag="Sparkle_US">(("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</gnip:matching_rule>  </gnip:matching_rules></entry>
-<entry xmlns:gnip="http://www.gnip.com/schemas/2010" xmlns="http://www.w3.org/2005/Atom">  <id>9561929525</id>  <published>2013-08-21T15:46:26Z</published>  <updated>2013-08-21T15:46:26Z</updated>  <title type="html">Torrit posted an image to Flickr</title>  <summary type="text">wedding hartlepool hart village staincliffe hotel</summary>  <category term="ImagePosted" label="Image Posted"/>  <link rel="alternate" href="http://www.flickr.com/photos/58789603@N02/9561929525"/>  <source>    <link rel="self" href="http://com.facebook.api.flickr.com/services/rest/?method=flickr.photos.search&amp;text=%28%28%22sparkle%20papertowel%22%20OR%20%22paper%20towel%20sparkle%22%20OR%20%22paper%20towels%20sparkle%22%20OR%20%22paper-towel%20sparkle%22%20OR%20%22paper-towels%20sparkle%22%20OR%20%22papertowels%20sparkle%22%20OR%20%22disposable%20towel%20sparkle%22%20OR%20%22disposable%20towels%20sparkle%22%20OR%20%22towel%20sparkle%22%20OR%20%22towels%20sparkle%22%20OR%20%22b&amp;extras=date_upload,own
 er_name,tags,geo,machine_tags,description&amp;per_page=100&amp;api_key=361f14d394043a9eeca7cfc44e164d03&amp;min_upload_date=Wed+Aug+21+15%3A37%3A53+UTC+2013"/>    <title>Flickr - Keyword - Search - (("sparkle papertowel" OR "paper towel sparkle" OR "paper towels sparkle" OR "paper-towel sparkle" OR "paper-towels sparkle" OR "papertowels sparkle" OR "disposable towel sparkle" OR "disposable towels sparkle" OR "towel sparkle" OR "towels sparkle" OR "b</title>    <updated>2013-08-21T08-56-24Z</updated>  

<TRUNCATED>

[05/15] git commit: STREAMS-46 providers

Posted by sb...@apache.org.
STREAMS-46 providers


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

Branch: refs/heads/master
Commit: a03120223def4bde75c0d0c11c7a7214651a593f
Parents: 3f6a015
Author: sblackmon <sb...@w2odigital.com>
Authored: Mon Jul 7 07:15:43 2014 -0700
Committer: sblackmon <sb...@w2odigital.com>
Committed: Mon Jul 21 10:30:45 2014 -0500

----------------------------------------------------------------------
 .../provider/FacebookFriendFeedProvider.java    | 285 +++++++++++++++++++
 .../provider/FacebookFriendUpdatesProvider.java | 285 +++++++++++++++++++
 .../FacebookUserInformationProvider.java        |  57 +++-
 .../FacebookUserInformationConfiguration.json   |   5 +
 4 files changed, 618 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/a0312022/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
new file mode 100644
index 0000000..a66d213
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
@@ -0,0 +1,285 @@
+/*
+ * 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 com.facebook.provider;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Queues;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookFriendFeedProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookFriendFeedProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendFeedProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
+
+    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
+
+    protected Iterator<String[]> idsBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookFriendFeedProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookFriendFeedProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        shutdownAndAwaitTermination(executor);
+        running.set(true);
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        StreamsResultSet current;
+
+        synchronized (FacebookUserstreamProvider.class) {
+            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
+            current.setCounter(new DatumStatusCounter());
+            current.getCounter().add(countersCurrent);
+            countersTotal.add(countersCurrent);
+            countersCurrent = new DatumStatusCounter();
+            providerQueue.clear();
+        }
+
+        return current;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+        Facebook client = getFacebookClient();
+
+        try {
+            ResponseList<Friend> friendResponseList = client.friends().getFriends();
+            Paging<Friend> friendPaging;
+            do {
+
+                for( Friend friend : friendResponseList ) {
+
+                    executor.submit(new FacebookFriendFeedTask(this, friend.getId()));
+                }
+                friendPaging = friendResponseList.getPaging();
+                friendResponseList = client.fetchNext(friendPaging);
+            } while( friendPaging != null &&
+                    friendResponseList != null );
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(configuration.getOauth().getAppId())
+            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFriendFeedTask implements Runnable {
+
+        FacebookFriendFeedProvider provider;
+        Facebook client;
+        String id;
+
+        public FacebookFriendFeedTask(FacebookFriendFeedProvider provider, String id) {
+            this.provider = provider;
+            this.id = id;
+        }
+
+        @Override
+        public void run() {
+            client = provider.getFacebookClient();
+                try {
+                    ResponseList<Post> postResponseList = client.getFeed(id);
+                    Paging<Post> postPaging;
+                    do {
+
+                        for (Post item : postResponseList) {
+                            String json = DataObjectFactory.getRawJSON(item);
+                            org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                            try {
+                                lock.readLock().lock();
+                                ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                                countersCurrent.incrementAttempt();
+                            } finally {
+                                lock.readLock().unlock();
+                            }
+                        }
+                        postPaging = postResponseList.getPaging();
+                        postResponseList = client.fetchNext(postPaging);
+                    } while( postPaging != null &&
+                            postResponseList != null );
+
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/a0312022/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
new file mode 100644
index 0000000..a111823
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
@@ -0,0 +1,285 @@
+/*
+ * 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 com.facebook.provider;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigRenderOptions;
+import facebook4j.*;
+import facebook4j.conf.ConfigurationBuilder;
+import facebook4j.json.DataObjectFactory;
+import org.apache.streams.config.StreamsConfigurator;
+import org.apache.streams.core.DatumStatusCounter;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsProvider;
+import org.apache.streams.core.StreamsResultSet;
+import org.apache.streams.facebook.FacebookUserInformationConfiguration;
+import org.apache.streams.facebook.FacebookUserstreamConfiguration;
+import org.apache.streams.jackson.StreamsJacksonMapper;
+import org.apache.streams.util.ComponentUtils;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FacebookFriendUpdatesProvider implements StreamsProvider, Serializable
+{
+
+    public static final String STREAMS_ID = "FacebookFriendPostsProvider";
+    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendUpdatesProvider.class);
+
+    private static final ObjectMapper mapper = new StreamsJacksonMapper();
+
+    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private FacebookUserstreamConfiguration configuration;
+
+    private Class klass;
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
+
+    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
+
+    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
+
+    protected Iterator<String[]> idsBatches;
+
+    protected ExecutorService executor;
+
+    protected DateTime start;
+    protected DateTime end;
+
+    protected final AtomicBoolean running = new AtomicBoolean();
+
+    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
+    private DatumStatusCounter countersTotal = new DatumStatusCounter();
+
+    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
+        return new ThreadPoolExecutor(nThreads, nThreads,
+                5000L, TimeUnit.MILLISECONDS,
+                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
+    }
+
+    public FacebookFriendUpdatesProvider() {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config) {
+        this.configuration = config;
+    }
+
+    public FacebookFriendUpdatesProvider(Class klass) {
+        Config config = StreamsConfigurator.config.getConfig("facebook");
+        FacebookUserInformationConfiguration configuration;
+        try {
+            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
+        } catch (IOException e) {
+            e.printStackTrace();
+            return;
+        }
+        this.klass = klass;
+    }
+
+    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config, Class klass) {
+        this.configuration = config;
+        this.klass = klass;
+    }
+
+    public Queue<StreamsDatum> getProviderQueue() {
+        return this.providerQueue;
+    }
+
+    @Override
+    public void startStream() {
+        running.set(true);
+    }
+
+    public StreamsResultSet readCurrent() {
+
+        Preconditions.checkArgument(idsBatches.hasNext());
+
+        LOGGER.info("readCurrent");
+
+        // return stuff
+
+        LOGGER.info("Finished.  Cleaning up...");
+
+        LOGGER.info("Providing {} docs", providerQueue.size());
+
+        StreamsResultSet result =  new StreamsResultSet(providerQueue);
+        running.set(false);
+
+        LOGGER.info("Exiting");
+
+        return result;
+
+    }
+
+    public StreamsResultSet readNew(BigInteger sequence) {
+        LOGGER.debug("{} readNew", STREAMS_ID);
+        throw new NotImplementedException();
+    }
+
+    public StreamsResultSet readRange(DateTime start, DateTime end) {
+        LOGGER.debug("{} readRange", STREAMS_ID);
+        this.start = start;
+        this.end = end;
+        readCurrent();
+        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
+        return result;
+    }
+
+    @Override
+    public boolean isRunning() {
+        return running.get();
+    }
+
+    void shutdownAndAwaitTermination(ExecutorService pool) {
+        pool.shutdown(); // Disable new tasks from being submitted
+        try {
+            // Wait a while for existing tasks to terminate
+            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
+                pool.shutdownNow(); // Cancel currently executing tasks
+                // Wait a while for tasks to respond to being cancelled
+                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
+                    System.err.println("Pool did not terminate");
+            }
+        } catch (InterruptedException ie) {
+            // (Re-)Cancel if current thread also interrupted
+            pool.shutdownNow();
+            // Preserve interrupt status
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public void prepare(Object o) {
+
+        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
+
+        Preconditions.checkNotNull(providerQueue);
+        Preconditions.checkNotNull(this.klass);
+        Preconditions.checkNotNull(configuration.getOauth().getAppId());
+        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
+        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
+
+        Facebook client = getFacebookClient();
+
+        try {
+            ResponseList<Friend> friendResponseList = client.friends().getFriends();
+            Paging<Friend> friendPaging;
+            do {
+
+                for( Friend friend : friendResponseList ) {
+
+                    //client.rawAPI().callPostAPI();
+                    // add a subscription
+                }
+                friendPaging = friendResponseList.getPaging();
+                friendResponseList = client.fetchNext(friendPaging);
+            } while( friendPaging != null &&
+                    friendResponseList != null );
+        } catch (FacebookException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    protected Facebook getFacebookClient()
+    {
+        ConfigurationBuilder cb = new ConfigurationBuilder();
+        cb.setDebugEnabled(true)
+            .setOAuthAppId(configuration.getOauth().getAppId())
+            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
+            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
+            .setOAuthPermissions(ALL_PERMISSIONS)
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
+
+        FacebookFactory ff = new FacebookFactory(cb.build());
+        Facebook facebook = ff.getInstance();
+
+        return facebook;
+    }
+
+    @Override
+    public void cleanUp() {
+        shutdownAndAwaitTermination(executor);
+    }
+
+    private class FacebookFeedPollingTask implements Runnable {
+
+        FacebookUserstreamProvider provider;
+        Facebook client;
+
+        private Set<Post> priorPollResult = Sets.newHashSet();
+
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
+            provider = facebookUserstreamProvider;
+        }
+
+        @Override
+        public void run() {
+            client = provider.getFacebookClient();
+            while (provider.isRunning()) {
+                try {
+                    ResponseList<Post> postResponseList = client.getHome();
+                    Set<Post> update = Sets.newHashSet(postResponseList);
+                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
+                    Set<Post> entrySet = Sets.difference(update, repeats);
+                    for (Post item : entrySet) {
+                        String json = DataObjectFactory.getRawJSON(item);
+                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
+                        try {
+                            lock.readLock().lock();
+                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
+                            countersCurrent.incrementAttempt();
+                        } finally {
+                            lock.readLock().unlock();
+                        }
+                    }
+                    priorPollResult = update;
+                    Thread.sleep(configuration.getPollIntervalMillis());
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/a0312022/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
index a167947..8640f5d 100644
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
@@ -140,24 +140,52 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
             e.printStackTrace();
         }
 
-        while( idsBatches.hasNext() ) {
-            try {
-                List<User> userList = client.users().getUsers(idsBatches.next());
-                for (User user : userList) {
-
-                    try {
-                        String json = mapper.writeValueAsString(user);
-                        providerQueue.add(
-                            new StreamsDatum(json, DateTime.now())
-                        );
-                    } catch (JsonProcessingException e) {
-                        //                        e.printStackTrace();
+        if( idsBatches.hasNext()) {
+            while (idsBatches.hasNext()) {
+                try {
+                    List<User> userList = client.users().getUsers(idsBatches.next());
+                    for (User user : userList) {
+
+                        try {
+                            String json = mapper.writeValueAsString(user);
+                            providerQueue.add(
+                                    new StreamsDatum(json, DateTime.now())
+                            );
+                        } catch (JsonProcessingException e) {
+                            //                        e.printStackTrace();
+                        }
                     }
-                }
 
+                } catch (FacebookException e) {
+                    e.printStackTrace();
+                }
+            }
+        } else {
+            try {
+                ResponseList<Friend> friendResponseList = client.friends().getFriends();
+                Paging<Friend> friendPaging;
+                do {
+
+                    for( Friend friend : friendResponseList ) {
+
+                        String json;
+                        try {
+                            json = mapper.writeValueAsString(friend);
+                            providerQueue.add(
+                                    new StreamsDatum(json)
+                            );
+                        } catch (JsonProcessingException e) {
+//                        e.printStackTrace();
+                        }
+                    }
+                    friendPaging = friendResponseList.getPaging();
+                    friendResponseList = client.fetchNext(friendPaging);
+                } while( friendPaging != null &&
+                        friendResponseList != null );
             } catch (FacebookException e) {
                 e.printStackTrace();
             }
+
         }
 
         LOGGER.info("Finished.  Cleaning up...");
@@ -254,7 +282,8 @@ public class FacebookUserInformationProvider implements StreamsProvider, Seriali
             .setOAuthAppSecret(facebookUserInformationConfiguration.getOauth().getAppSecret())
             .setOAuthAccessToken(facebookUserInformationConfiguration.getOauth().getUserAccessToken())
             .setOAuthPermissions(ALL_PERMISSIONS)
-            .setJSONStoreEnabled(true);
+            .setJSONStoreEnabled(true)
+            .setClientVersion("v1.0");
 
         FacebookFactory ff = new FacebookFactory(cb.build());
         Facebook facebook = ff.getInstance();

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/a0312022/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
index 0454178..b351be9 100644
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json
@@ -13,6 +13,11 @@
             "items": {
                 "type": "string"
             }
+        },
+        "pollIntervalMillis": {
+            "type": "integer",
+            "default" : "60000",
+            "description": "Polling interval in ms"
         }
      }
 }
\ No newline at end of file


[12/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/graph/Post.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/graph/Post.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/graph/Post.json
new file mode 100644
index 0000000..23bcb08
--- /dev/null
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/org/apache/streams/facebook/graph/Post.json
@@ -0,0 +1,203 @@
+{
+    "type": "object",
+    "$schema": "http://json-schema.org/draft-03/schema",
+    "id": "#",
+    "javaType": "org.apache.streams.facebook.Post",
+    "properties": {
+        "id": {
+            "type": "string"
+        },
+        "from": {
+            "type": "object",
+            "properties": {
+                "id": {
+                    "type": "string"
+                },
+                "name": {
+                    "type": "string"
+                }
+            }
+        },
+        "to": {
+            "type": "object",
+            "properties": {
+                "data": {
+                    "type": "array",
+                    "items": {
+                        "type": "object",
+                        "properties": {
+                            "id": {
+                                "type": "string"
+                            },
+                            "name": {
+                                "type": "string"
+                            }
+                        }
+                    }
+                }
+            }
+        },
+        "message": {
+            "type": "string"
+        },
+        "message_tags": {
+            "type": "object",
+            "properties": {
+                "data": {
+                    "type": "array",
+                    "items": {
+                        "type": "object",
+                        "properties": {
+                            "id": {
+                                "type": "string"
+                            },
+                            "name": {
+                                "type": "string"
+                            }
+                        }
+                    }
+                }
+            }
+        },
+        "picture": {
+            "type": "string"
+        },
+        "link": {
+            "type": "string"
+        },
+        "name": {
+            "type": "string"
+        },
+        "caption": {
+            "type": "string"
+        },
+        "description": {
+            "type": "string"
+        },
+        "source": {
+            "type": "string"
+        },
+        "icon": {
+            "type": "string"
+        },
+        "actions": {
+            "type": "array",
+            "items": {
+                "type": "object",
+                "properties": {
+                    "name": {
+                        "type": "string"
+                    },
+                    "link": {
+                        "type": "string"
+                    }
+                }
+            }
+        },
+        "comments": {
+            "type": "array",
+            "items": {
+                "type": "object",
+                "properties": {
+                    "id": {
+                        "type": "string"
+                    },
+                    "from": {
+                        "type": "string"
+                    },
+                    "message": {
+                        "type": "string"
+                    },
+                    "created_time": {
+                        "type": "string",
+                        "format" : "date-time"
+                    }
+                }
+            }
+        },
+        "likes": {
+            "type": "array",
+            "items": {
+                "type": "object",
+                "properties": {
+                    "name": {
+                        "type": "string"
+                    },
+                    "link": {
+                        "type": "string"
+                    }
+                }
+            }
+        },
+        "type": {
+            "type": "string"
+        },
+        "place": {
+            "type": "object",
+            "properties": {
+                "name": {
+                    "type": "string"
+                },
+                "id": {
+                    "type": "string"
+                }
+            }
+        },
+        "story": {
+            "type": "string"
+        },
+        "shares": {
+            "type": "int"
+        },
+        "object_id": {
+            "type": "int"
+        },
+        "application": {
+            "type": "object",
+            "properties": {
+                "name": {
+                    "type": "string"
+                },
+                "id": {
+                    "type": "string"
+                }
+            }
+        },
+        "created_time": {
+            "type": "string",
+            "format" : "date-time"
+        },
+        "updated_time": {
+            "type": "string",
+            "format" : "date-time"
+        },
+        "include_hidden": {
+            "type": "boolean"
+        },
+        "status_type": {
+            "type": "string"
+        },
+        "properties": {
+            "type": "array",
+            "items": {
+                "type": "object",
+                "properties": {
+                    "name": {
+                        "type": "string"
+                    },
+                    "text": {
+                        "type": "string"
+                    }
+                }
+            }
+        },
+        "privacy": {
+            "type": "object",
+            "properties": {
+                "value": {
+                    "type": "string"
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
index 7f3ab27..4de9ab3 100644
--- a/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
+++ b/streams-contrib/streams-provider-facebook/src/test/java/org/apache/streams/facebook/test/FacebookActivitySerDeTest.java
@@ -18,18 +18,16 @@
 
 package org.apache.streams.facebook.test;
 
-import com.facebook.api.FacebookPostActivitySerializer;
+import org.apache.streams.facebook.api.FacebookPostActivitySerializer;
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Joiner;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.input.BoundedInputStream;
-import org.apache.streams.data.ActivitySerializer;
 import org.apache.streams.facebook.Post;
 import org.apache.streams.jackson.StreamsJacksonMapper;
 import org.apache.streams.pojo.json.Activity;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-gnip/gnip-edc-facebook/pom.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-gnip/gnip-edc-facebook/pom.xml b/streams-contrib/streams-provider-gnip/gnip-edc-facebook/pom.xml
index e0603bd..7965d48 100644
--- a/streams-contrib/streams-provider-gnip/gnip-edc-facebook/pom.xml
+++ b/streams-contrib/streams-provider-gnip/gnip-edc-facebook/pom.xml
@@ -105,7 +105,7 @@
                     <encoding>${project.build.sourceEncoding}</encoding>
                     <forceRegenerate>true</forceRegenerate>
                     <removeOldOutput>false</removeOldOutput>
-                    <generatePackage>com.facebook.api</generatePackage>
+                    <generatePackage>org.apache.streams.facebook.api</generatePackage>
                     <plugins>
                         <plugin>
                             <groupId>org.jvnet.jaxb2_commons</groupId>


[06/15] git commit: STREAMS-105 | Updated the InstagramTypeConverter to use the conversion utility functions provided in InstagramActivityUtil

Posted by sb...@apache.org.
STREAMS-105 | Updated the InstagramTypeConverter to use the conversion utility functions provided in InstagramActivityUtil


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

Branch: refs/heads/master
Commit: e8511ada025c1837db8ddd4fb62d24eb231a9f7f
Parents: a031202
Author: Robert Douglas <rd...@w2odigital.com>
Authored: Wed Jul 2 10:48:35 2014 -0500
Committer: sblackmon <sb...@w2odigital.com>
Committed: Mon Jul 21 12:25:43 2014 -0500

----------------------------------------------------------------------
 .../provider/FacebookUserstreamProvider.java    | 72 +++++++++++++-------
 .../FacebookUserstreamConfiguration.json        |  7 ++
 2 files changed, 53 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/e8511ada/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
index eae8069..af7868b 100644
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
+++ b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
@@ -62,7 +62,7 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
 
     private static final ObjectMapper mapper = new StreamsJacksonMapper();
 
-    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
+    private static final String ALL_PERMISSIONS = "read_stream";
     private FacebookUserstreamConfiguration configuration;
 
     private Class klass;
@@ -88,6 +88,8 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
     private DatumStatusCounter countersCurrent = new DatumStatusCounter();
     private DatumStatusCounter countersTotal = new DatumStatusCounter();
 
+    protected Facebook client;
+
     private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
         return new ThreadPoolExecutor(nThreads, nThreads,
                 5000L, TimeUnit.MILLISECONDS,
@@ -133,31 +135,17 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
     @Override
     public void startStream() {
 
-        executor.submit(new FacebookFeedPollingTask(this));
-        running.set(true);
-    }
-
-    private void loadBatch(String[] ids) {
-        Facebook client = getFacebookClient();
-        int keepTrying = 0;
-
-        // keep trying to load, give it 5 attempts.
-        //while (keepTrying < 10)
-        while (keepTrying < 1) {
-            try {
-                for (User user : client.getUsers(ids)) {
-                    String json = DataObjectFactory.getRawJSON(user);
+        client = getFacebookClient();
 
-                    providerQueue.offer(new StreamsDatum(json));
-//
-                }
-                ;
-            } catch (FacebookException e) {
-                e.printStackTrace();
-                return;
+        if( configuration.getInfo() != null &&
+            configuration.getInfo().size() > 0 ) {
+            for( String id : configuration.getInfo()) {
+                executor.submit(new FacebookFeedPollingTask(this, id));
             }
-
+        } else {
+            executor.submit(new FacebookFeedPollingTask(this));
         }
+        running.set(true);
     }
 
     public StreamsResultSet readCurrent() {
@@ -225,6 +213,28 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
         Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
         Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
 
+        client = getFacebookClient();
+
+        if( configuration.getInfo() != null &&
+            configuration.getInfo().size() > 0 ) {
+
+            List<String> ids = new ArrayList<String>();
+            List<String[]> idsBatches = new ArrayList<String[]>();
+
+            for (String s : configuration.getInfo()) {
+                if (s != null) {
+                    ids.add(s);
+
+                    if (ids.size() >= 100) {
+                        // add the batch
+                        idsBatches.add(ids.toArray(new String[ids.size()]));
+                        // reset the Ids
+                        ids = new ArrayList<String>();
+                    }
+
+                }
+            }
+        }
     }
 
     protected Facebook getFacebookClient() {
@@ -251,19 +261,29 @@ public class FacebookUserstreamProvider implements StreamsProvider, Serializable
 
         FacebookUserstreamProvider provider;
         Facebook client;
+        String id;
 
         private Set<Post> priorPollResult = Sets.newHashSet();
 
         public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
-            provider = facebookUserstreamProvider;
+            this.provider = facebookUserstreamProvider;
         }
 
+        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider, String id) {
+            this.provider = facebookUserstreamProvider;
+            this.client = provider.client;
+            this.id = id;
+        }
         @Override
         public void run() {
-            client = provider.getFacebookClient();
             while (provider.isRunning()) {
+                ResponseList<Post> postResponseList;
                 try {
-                    ResponseList<Post> postResponseList = client.getHome();
+                    if( id != null )
+                        postResponseList = client.getFeed(id);
+                    else
+                        postResponseList = client.getHome();
+
                     Set<Post> update = Sets.newHashSet(postResponseList);
                     Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
                     Set<Post> entrySet = Sets.difference(update, repeats);

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/e8511ada/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
index c823a12..bcb2258 100644
--- a/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
+++ b/streams-contrib/streams-provider-facebook/src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json
@@ -6,6 +6,13 @@
     "javaInterfaces": ["java.io.Serializable"],
     "extends": {"$ref":"FacebookConfiguration.json"},
     "properties": {
+        "info": {
+            "type": "array",
+            "description": "A list of user IDs, indicating users of interest",
+            "items": {
+                "type": "string"
+            }
+        },
         "pollIntervalMillis": {
             "type": "integer",
             "default" : "60000",


[15/15] git commit: Merge branch 'STREAMS-46'

Posted by sb...@apache.org.
Merge branch 'STREAMS-46'


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

Branch: refs/heads/master
Commit: d475b913d19587f3edc9dda5780b3846a9faf5f9
Parents: d309a59 dc432af
Author: sblackmon <sb...@apache.org>
Authored: Mon Aug 11 17:48:21 2014 -0500
Committer: sblackmon <sb...@apache.org>
Committed: Mon Aug 11 17:48:21 2014 -0500

----------------------------------------------------------------------
 .../streams-provider-facebook/pom.xml           |  22 +-
 .../api/FacebookPostActivitySerializer.java     | 334 -------------------
 ...FacebookPublicFeedXmlActivitySerializer.java |  29 --
 .../api/FacebookPostActivitySerializer.java     | 286 ++++++++++++++++
 ...FacebookPublicFeedXmlActivitySerializer.java |  29 ++
 .../processor/FacebookTypeConverter.java        | 194 +++++++++++
 .../provider/FacebookFriendFeedProvider.java    | 282 ++++++++++++++++
 .../provider/FacebookFriendUpdatesProvider.java | 286 ++++++++++++++++
 .../FacebookUserInformationProvider.java        | 299 +++++++++++++++++
 .../provider/FacebookUserstreamProvider.java    | 320 ++++++++++++++++++
 .../jsonschema/com/facebook/graph/Post.json     | 192 -----------
 .../streams/facebook/FacebookConfiguration.json |  49 +++
 .../FacebookUserInformationConfiguration.json   |  23 ++
 .../FacebookUserstreamConfiguration.json        |  22 ++
 .../org/apache/streams/facebook/graph/Post.json | 203 +++++++++++
 .../test/FacebookActivitySerDeTest.java         |  78 +++++
 .../FacebookPostActivitySerializerTest.java     | 215 ------------
 .../facebook/test/FacebookPostSerDeTest.java    |   6 +-
 .../src/test/resources/Facebook.json            | 250 ++++++++++++++
 .../org/apache/streams/data/Facebook.json       | 251 --------------
 .../gnip-edc-facebook/pom.xml                   |   2 +-
 .../src/test/resources/FlickrEDC.xml            | 128 +++----
 .../src/test/resources/RedditEDC.xml            | 200 +++++------
 .../src/test/resources/RedditEDCFlattened.xml   | 200 +++++------
 .../src/test/resources/redditTest.xml           |   2 +-
 25 files changed, 2611 insertions(+), 1291 deletions(-)
----------------------------------------------------------------------



[14/15] git commit: updated packages

Posted by sb...@apache.org.
updated packages


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

Branch: refs/heads/master
Commit: dc432af2c70e9ac443af3cbe57a7772fdd1e146d
Parents: e0cb5ec
Author: sblackmon <sb...@apache.org>
Authored: Mon Aug 11 17:03:50 2014 -0500
Committer: sblackmon <sb...@apache.org>
Committed: Mon Aug 11 17:03:50 2014 -0500

----------------------------------------------------------------------
 .../streams-provider-facebook/pom.xml           |   8 +-
 .../api/FacebookPostActivitySerializer.java     | 300 -----------------
 ...FacebookPublicFeedXmlActivitySerializer.java |  29 --
 .../processor/FacebookTypeConverter.java        | 194 -----------
 .../provider/FacebookFriendFeedProvider.java    | 285 -----------------
 .../provider/FacebookFriendUpdatesProvider.java | 285 -----------------
 .../FacebookUserInformationProvider.java        | 298 -----------------
 .../provider/FacebookUserstreamProvider.java    | 319 ------------------
 .../api/FacebookPostActivitySerializer.java     | 286 +++++++++++++++++
 ...FacebookPublicFeedXmlActivitySerializer.java |  29 ++
 .../processor/FacebookTypeConverter.java        | 194 +++++++++++
 .../provider/FacebookFriendFeedProvider.java    | 282 ++++++++++++++++
 .../provider/FacebookFriendUpdatesProvider.java | 286 +++++++++++++++++
 .../FacebookUserInformationProvider.java        | 299 +++++++++++++++++
 .../provider/FacebookUserstreamProvider.java    | 320 +++++++++++++++++++
 .../com/facebook/FacebookConfiguration.json     |  49 ---
 .../FacebookUserInformationConfiguration.json   |  23 --
 .../FacebookUserstreamConfiguration.json        |  22 --
 .../jsonschema/com/facebook/graph/Post.json     | 203 ------------
 .../streams/facebook/FacebookConfiguration.json |  49 +++
 .../FacebookUserInformationConfiguration.json   |  23 ++
 .../FacebookUserstreamConfiguration.json        |  22 ++
 .../org/apache/streams/facebook/graph/Post.json | 203 ++++++++++++
 .../test/FacebookActivitySerDeTest.java         |   4 +-
 .../gnip-edc-facebook/pom.xml                   |   2 +-
 .../src/test/resources/FlickrEDC.xml            | 128 ++++----
 .../src/test/resources/RedditEDC.xml            | 200 ++++++------
 .../src/test/resources/RedditEDCFlattened.xml   | 200 ++++++------
 .../src/test/resources/redditTest.xml           |   2 +-
 29 files changed, 2264 insertions(+), 2280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/pom.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/pom.xml b/streams-contrib/streams-provider-facebook/pom.xml
index 34d53a9..d688a00 100644
--- a/streams-contrib/streams-provider-facebook/pom.xml
+++ b/streams-contrib/streams-provider-facebook/pom.xml
@@ -101,10 +101,10 @@
                     <addCompileSourceRoot>true</addCompileSourceRoot>
                     <generateBuilders>true</generateBuilders>
                     <sourcePaths>
-                        <sourcePath>src/main/jsonschema/com/facebook/FacebookConfiguration.json</sourcePath>
-                        <sourcePath>src/main/jsonschema/com/facebook/FacebookUserInformationConfiguration.json</sourcePath>
-                        <sourcePath>src/main/jsonschema/com/facebook/FacebookUserstreamConfiguration.json</sourcePath>
-                        <sourcePath>src/main/jsonschema/com/facebook/graph/Post.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/org/apache/streams/facebook/FacebookConfiguration.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/org/apache/streams/facebook/FacebookUserInformationConfiguration.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/org/apache/streams/facebook/FacebookUserstreamConfiguration.json</sourcePath>
+                        <sourcePath>src/main/jsonschema/org/apache/streams/facebook/graph/Post.json</sourcePath>
                     </sourcePaths>
                     <outputDirectory>target/generated-sources/jsonschema2pojo</outputDirectory>
                     <targetPackage>com.facebook</targetPackage>

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
deleted file mode 100644
index 71bc5c9..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/api/FacebookPostActivitySerializer.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.api;
-
-
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.fasterxml.jackson.databind.ser.std.StdSerializer;
-import com.fasterxml.jackson.datatype.joda.JodaModule;
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.streams.data.ActivitySerializer;
-import org.apache.streams.exceptions.ActivitySerializerException;
-import org.apache.streams.facebook.Post;
-import org.apache.streams.pojo.json.*;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.joda.time.DateTime;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-import org.joda.time.format.ISODateTimeFormat;
-
-import java.io.IOException;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.*;
-import java.util.List;
-
-import static org.apache.streams.data.util.ActivityUtil.*;
-import static org.apache.streams.data.util.JsonUtil.jsonToJsonNode;
-
-/**
- * Serializes activity posts
- *   sblackmon: This class needs a rewrite
- */
-public class FacebookPostActivitySerializer implements ActivitySerializer<org.apache.streams.facebook.Post> {
-
-    public static final DateTimeFormatter FACEBOOK_FORMAT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ssZ");
-    public static final DateTimeFormatter ACTIVITY_FORMAT = ISODateTimeFormat.basicDateTime();
-
-    public static final String PROVIDER_NAME = "facebook";
-
-    public static ObjectMapper mapper;
-    static {
-        mapper = StreamsJacksonMapper.getInstance();
-    }
-
-    @Override
-    public String serializationFormat() {
-        return "facebook_post_json_v1";
-    }
-
-    @Override
-    public Post serialize(Activity deserialized) throws ActivitySerializerException {
-        throw new NotImplementedException("Not currently supported by this deserializer");
-    }
-
-    @Override
-    public Activity deserialize(Post post) throws ActivitySerializerException {
-        Activity activity = new Activity();
-        activity.setPublished(post.getCreatedTime());
-        activity.setUpdated(post.getUpdatedTime());
-        addActor(activity, mapper.convertValue(post.getFrom(), ObjectNode.class));
-        setProvider(activity);
-        setObjectType(post.getType(), activity);
-        parseObject(activity, mapper.convertValue(post, ObjectNode.class));
-        fixObjectId(activity);
-        fixContentFromSummary(activity);
-        return activity;
-    }
-
-    @Override
-    public List<Activity> deserializeAll(List<Post> serializedList) {
-        throw new NotImplementedException("Not currently supported by this deserializer");
-    }
-
-    private void fixContentFromSummary(Activity activity) {
-        //we MUST have a content field set, so choose the best option
-        if(activity.getContent() == null) {
-            activity.setContent(activity.getAdditionalProperties().containsKey("summary") ?
-                    (String) activity.getAdditionalProperties().get("summary") :
-                    activity.getObject().getSummary());
-        }
-    }
-
-    private void fixObjectId(Activity activity) {
-        //An artifact of schema generation, the default value is {link}
-        if(activity.getObject().getId().equals("{link}")) {
-            activity.getObject().setId(null);
-        }
-    }
-
-    private void setObjectType(String type, Activity activity) {
-        ActivityObject object = new ActivityObject();
-        activity.setObject(object);
-        object.setObjectType(type);
-    }
-
-    private void setProvider(Activity activity) {
-        Provider provider = new Provider();
-        provider.setId(getProviderId(PROVIDER_NAME));
-        provider.setDisplayName(PROVIDER_NAME);
-        activity.setProvider(provider);
-    }
-
-    private String getObjectType(JsonNode node) {
-        Iterator<Map.Entry<String, JsonNode>> fields = node.fields();
-        ensureMoreFields(fields);
-        Map.Entry<String, JsonNode> field = fields.next();
-        //ensureNoMoreFields(fields);
-        return node.asText();
-    }
-
-    private void parseObject(Activity activity, JsonNode jsonNode) throws ActivitySerializerException {
-        for(Iterator<Map.Entry<String, JsonNode>> fields = jsonNode.fields(); fields.hasNext();) {
-            Map.Entry<String, JsonNode> field = fields.next();
-            String key = field.getKey();
-            JsonNode value = field.getValue();
-            mapField(activity, key, value);
-        }
-    }
-
-    private void mapField(Activity activity, String name, JsonNode value) throws ActivitySerializerException {
-        if("application".equals(name)) {
-            addGenerator(activity, value);
-        } else if ("caption".equals(name)) {
-            addSummary(activity, value);
-        } else if ("comments".equals(name)) {
-            addAttachments(activity, value);
-        } else if ("description".equals(name)) {
-            addObjectSummary(activity, value);
-        } else if ("from".equals(name)) {
-            addActor(activity, value);
-        } else if ("icon".equals(name)) {
-            addIcon(activity, value);
-        } else if ("id".equals(name)) {
-            addId(activity, value);
-        } else if ("is_hidden".equals(name)) {
-            addObjectHiddenExtension(activity, value);
-        } else if ("like_count".equals(name)) {
-            addLikeExtension(activity, value);
-        } else if ("link".equals(name)) {
-            addObjectLink(activity, value);
-        } else if ("message".equals(name)) {
-            activity.setContent(value.asText());
-        } else if ("name".equals(name)) {
-            addObjectName(activity, value);
-        } else if ("object_id".equals(name)) {
-            addObjectId(activity, value);
-        } else if ("picture".equals(name)) {
-            addObjectImage(activity, value);
-        } else if ("place".equals(name)) {
-            addLocationExtension(activity, value);
-        } else if ("shares".equals(name)) {
-            addRebroadcastExtension(activity, value);
-        } else if ("source".equals(name)) {
-            addObjectLink(activity, value);
-        } else if ("story".equals(name)) {
-            addTitle(activity, value);
-        }
-    }
-
-    private void addSummary(Activity activity, JsonNode value) {
-        activity.setAdditionalProperty("summary", value.asText());
-    }
-
-    private void addTitle(Activity activity, JsonNode value) {
-        activity.setTitle(value.asText());
-    }
-
-    private void addLikeExtension(Activity activity, JsonNode value) {
-        Map<String, Object> extensions = ensureExtensions(activity);
-        extensions.put(LIKES_EXTENSION, value.asInt());
-    }
-
-    private void addLocationExtension(Activity activity, JsonNode value) {
-        Map<String, Object> extensions = ensureExtensions(activity);
-        if(value.has("location")) {
-            Map<String, Object> location = new HashMap<String, Object>();
-            JsonNode fbLocation = value.get("location");
-            if(fbLocation.has("country")) {
-                location.put(LOCATION_EXTENSION_COUNTRY, fbLocation.get("country"));
-            }
-            if(fbLocation.has("latitude") && fbLocation.has("longitude")) {
-                location.put(LOCATION_EXTENSION_COORDINATES, String.format("%s,%s", fbLocation.get("longitude"), fbLocation.get("latitude")));
-            }
-            extensions.put(LOCATION_EXTENSION, location);
-        }
-    }
-
-    private void addObjectImage(Activity activity, JsonNode value) {
-        Image image = new Image();
-        image.setUrl(value.asText());
-        activity.getObject().setImage(image);
-    }
-
-    private void addObjectId(Activity activity, JsonNode value) {
-        activity.getObject().setId(getObjectId("facebook", activity.getObject().getObjectType(), value.asText()));
-    }
-
-    private void addObjectName(Activity activity, JsonNode value) {
-        activity.getObject().setDisplayName(value.asText());
-    }
-
-    private void addId(Activity activity, JsonNode value) {
-        activity.setId(getActivityId(PROVIDER_NAME, value.asText()));
-    }
-
-    private void addObjectLink(Activity activity, JsonNode value) {
-        activity.getObject().setUrl(value.asText());
-    }
-
-    private void addRebroadcastExtension(Activity activity, JsonNode value) {
-        Map<String, Object> extensions = ensureExtensions(activity);
-        if(value.has("count")) {
-            extensions.put(REBROADCAST_EXTENSION, value.get("count").asInt());
-        }
-    }
-
-    private void addObjectHiddenExtension(Activity activity, JsonNode value) {
-        Map<String, Object> extensions = ensureExtensions(activity);
-        extensions.put("hidden", value.asBoolean());
-    }
-
-    private void addIcon(Activity activity, JsonNode value) {
-        Icon icon = new Icon();
-        //Apparently the Icon didn't map from the schema very well
-        icon.setAdditionalProperty("url", value.asText());
-        activity.setIcon(icon);
-    }
-
-    private void addActor(Activity activity, JsonNode value) {
-        Actor actor = new Actor();
-        if(value.has("name")) {
-            actor.setDisplayName(value.get("name").asText());
-        }
-        if(value.has("id")) {
-            actor.setId(getPersonId(PROVIDER_NAME, value.get("id").asText()));
-        }
-        activity.setActor(actor);
-    }
-
-    private void addObjectSummary(Activity activity, JsonNode value) {
-        activity.getObject().setSummary(value.asText());
-    }
-
-    private void addGenerator(Activity activity, JsonNode value) {
-        Generator generator = new Generator();
-        if(value.has("id")) {
-            generator.setId(getObjectId(PROVIDER_NAME, "generator", value.get("id").asText()));
-        }
-        if(value.has("name")) {
-            generator.setDisplayName(value.get("name").asText());
-        }
-        if(value.has("namespace")) {
-            generator.setSummary(value.get("namespace").asText());
-        }
-        activity.setGenerator(generator);
-    }
-
-    private void addAttachments(Activity activity, JsonNode value) {
-        //No direct mapping at this time
-    }
-
-    private static void ensureMoreFields(Iterator<Map.Entry<String, JsonNode>> fields) {
-        if(!fields.hasNext()) {
-            throw new IllegalStateException("Facebook activity must have one and only one root element");
-        }
-    }
-    private static void ensureNoMoreFields(Iterator<Map.Entry<String, JsonNode>> fields) {
-        if(fields.hasNext()) {
-            throw new IllegalStateException("Facebook activity must have one and only one root element");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
deleted file mode 100644
index f126d88..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/feed/FacebookPublicFeedXmlActivitySerializer.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.feed;
-
-/**
- * Created with IntelliJ IDEA.
- * User: sblackmon
- * Date: 10/2/13
- * Time: 6:32 PM
- * To change this template use File | Settings | File Templates.
- */
-public class FacebookPublicFeedXmlActivitySerializer {
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
deleted file mode 100644
index 6ddb673..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/processor/FacebookTypeConverter.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.processor;
-
-import com.facebook.api.FacebookPostActivitySerializer;
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.collect.Lists;
-import org.apache.streams.core.StreamsDatum;
-import org.apache.streams.core.StreamsProcessor;
-import org.apache.streams.exceptions.ActivitySerializerException;
-import org.apache.streams.facebook.Post;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.pojo.json.Activity;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Queue;
-
-/**
- * Created by sblackmon on 12/10/13.
- */
-public class FacebookTypeConverter implements StreamsProcessor {
-
-    public final static String STREAMS_ID = "TwitterTypeConverter";
-
-    private final static Logger LOGGER = LoggerFactory.getLogger(FacebookTypeConverter.class);
-
-    private ObjectMapper mapper;
-
-    private Queue<StreamsDatum> inQueue;
-    private Queue<StreamsDatum> outQueue;
-
-    private Class inClass;
-    private Class outClass;
-
-    private FacebookPostActivitySerializer facebookPostActivitySerializer;
-
-    private int count = 0;
-
-    public final static String TERMINATE = new String("TERMINATE");
-
-    public FacebookTypeConverter(Class inClass, Class outClass) {
-        this.inClass = inClass;
-        this.outClass = outClass;
-    }
-
-    public Queue<StreamsDatum> getProcessorOutputQueue() {
-        return outQueue;
-    }
-
-    public void setProcessorInputQueue(Queue<StreamsDatum> inputQueue) {
-        inQueue = inputQueue;
-    }
-
-    public Object convert(ObjectNode event, Class inClass, Class outClass) throws ActivitySerializerException, JsonProcessingException {
-
-        Object result = null;
-
-        if( outClass.equals( Activity.class )) {
-            LOGGER.debug("ACTIVITY");
-            result = facebookPostActivitySerializer.deserialize(mapper.convertValue(event, Post.class));
-        } else if( outClass.equals( Post.class )) {
-            LOGGER.debug("POST");
-            result = mapper.convertValue(event, Post.class);
-        } else if( outClass.equals( ObjectNode.class )) {
-            LOGGER.debug("OBJECTNODE");
-            result = mapper.convertValue(event, ObjectNode.class);
-        }
-
-        // no supported conversion were applied
-        if( result != null ) {
-            count ++;
-            return result;
-        }
-
-        LOGGER.debug("CONVERT FAILED");
-
-        return null;
-
-    }
-
-    public boolean validate(Object document, Class klass) {
-
-        // TODO
-        return true;
-    }
-
-    public boolean isValidJSON(final String json) {
-        boolean valid = false;
-        try {
-            final JsonParser parser = new ObjectMapper().getJsonFactory()
-                    .createJsonParser(json);
-            while (parser.nextToken() != null) {
-            }
-            valid = true;
-        } catch (JsonParseException jpe) {
-            LOGGER.warn("validate: {}", jpe);
-        } catch (IOException ioe) {
-            LOGGER.warn("validate: {}", ioe);
-        }
-
-        return valid;
-    }
-
-    @Override
-    public List<StreamsDatum> process(StreamsDatum entry) {
-
-        StreamsDatum result = null;
-
-        try {
-
-            Object item = entry.getDocument();
-            ObjectNode node;
-
-            LOGGER.debug("{} processing {}", STREAMS_ID, item.getClass());
-
-            if( item instanceof String ) {
-
-                // if the target is string, just pass-through
-                if( String.class.equals(outClass)) {
-                    result = entry;
-                }
-                else {
-                    // first check for valid json
-                    node = (ObjectNode)mapper.readTree((String)item);
-
-                    // since data is coming from outside provider, we don't know what type the events are
-                    // for now we'll assume post
-
-                    Object out = convert(node, Post.class, outClass);
-
-                    if( out != null && validate(out, outClass))
-                        result = new StreamsDatum(out);
-                }
-
-            } else if( item instanceof ObjectNode || item instanceof Post) {
-
-                // first check for valid json
-                node = (ObjectNode)mapper.valueToTree(item);
-
-                // since data is coming from outside provider, we don't know what type the events are
-                // for now we'll assume post
-
-                Object out = convert(node, Post.class, outClass);
-
-                if( out != null && validate(out, outClass))
-                    result = new StreamsDatum(out);
-
-            }
-
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-
-        if( result != null )
-            return Lists.newArrayList(result);
-        else
-            return Lists.newArrayList();
-    }
-
-    @Override
-    public void prepare(Object o) {
-        mapper = new StreamsJacksonMapper();
-        facebookPostActivitySerializer = new FacebookPostActivitySerializer();
-    }
-
-    @Override
-    public void cleanUp() {
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
deleted file mode 100644
index a66d213..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendFeedProvider.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.provider;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigRenderOptions;
-import facebook4j.*;
-import facebook4j.conf.ConfigurationBuilder;
-import facebook4j.json.DataObjectFactory;
-import org.apache.streams.config.StreamsConfigurator;
-import org.apache.streams.core.DatumStatusCounter;
-import org.apache.streams.core.StreamsDatum;
-import org.apache.streams.core.StreamsProvider;
-import org.apache.streams.core.StreamsResultSet;
-import org.apache.streams.facebook.FacebookUserInformationConfiguration;
-import org.apache.streams.facebook.FacebookUserstreamConfiguration;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.util.ComponentUtils;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import sun.reflect.generics.reflectiveObjects.NotImplementedException;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigInteger;
-import java.util.Iterator;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class FacebookFriendFeedProvider implements StreamsProvider, Serializable
-{
-
-    public static final String STREAMS_ID = "FacebookFriendFeedProvider";
-    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendFeedProvider.class);
-
-    private static final ObjectMapper mapper = new StreamsJacksonMapper();
-
-    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
-    private FacebookUserstreamConfiguration configuration;
-
-    private Class klass;
-    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
-
-    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
-
-    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
-
-    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
-
-    protected Iterator<String[]> idsBatches;
-
-    protected ExecutorService executor;
-
-    protected DateTime start;
-    protected DateTime end;
-
-    protected final AtomicBoolean running = new AtomicBoolean();
-
-    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
-    private DatumStatusCounter countersTotal = new DatumStatusCounter();
-
-    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
-        return new ThreadPoolExecutor(nThreads, nThreads,
-                5000L, TimeUnit.MILLISECONDS,
-                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
-    }
-
-    public FacebookFriendFeedProvider() {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration configuration;
-        try {
-            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-    }
-
-    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config) {
-        this.configuration = config;
-    }
-
-    public FacebookFriendFeedProvider(Class klass) {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration configuration;
-        try {
-            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-        this.klass = klass;
-    }
-
-    public FacebookFriendFeedProvider(FacebookUserstreamConfiguration config, Class klass) {
-        this.configuration = config;
-        this.klass = klass;
-    }
-
-    public Queue<StreamsDatum> getProviderQueue() {
-        return this.providerQueue;
-    }
-
-    @Override
-    public void startStream() {
-        shutdownAndAwaitTermination(executor);
-        running.set(true);
-    }
-
-    public StreamsResultSet readCurrent() {
-
-        StreamsResultSet current;
-
-        synchronized (FacebookUserstreamProvider.class) {
-            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
-            current.setCounter(new DatumStatusCounter());
-            current.getCounter().add(countersCurrent);
-            countersTotal.add(countersCurrent);
-            countersCurrent = new DatumStatusCounter();
-            providerQueue.clear();
-        }
-
-        return current;
-
-    }
-
-    public StreamsResultSet readNew(BigInteger sequence) {
-        LOGGER.debug("{} readNew", STREAMS_ID);
-        throw new NotImplementedException();
-    }
-
-    public StreamsResultSet readRange(DateTime start, DateTime end) {
-        LOGGER.debug("{} readRange", STREAMS_ID);
-        this.start = start;
-        this.end = end;
-        readCurrent();
-        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
-        return result;
-    }
-
-    @Override
-    public boolean isRunning() {
-        return running.get();
-    }
-
-    void shutdownAndAwaitTermination(ExecutorService pool) {
-        pool.shutdown(); // Disable new tasks from being submitted
-        try {
-            // Wait a while for existing tasks to terminate
-            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
-                pool.shutdownNow(); // Cancel currently executing tasks
-                // Wait a while for tasks to respond to being cancelled
-                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
-                    System.err.println("Pool did not terminate");
-            }
-        } catch (InterruptedException ie) {
-            // (Re-)Cancel if current thread also interrupted
-            pool.shutdownNow();
-            // Preserve interrupt status
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public void prepare(Object o) {
-
-        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
-
-        Preconditions.checkNotNull(providerQueue);
-        Preconditions.checkNotNull(this.klass);
-        Preconditions.checkNotNull(configuration.getOauth().getAppId());
-        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
-        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
-
-        Facebook client = getFacebookClient();
-
-        try {
-            ResponseList<Friend> friendResponseList = client.friends().getFriends();
-            Paging<Friend> friendPaging;
-            do {
-
-                for( Friend friend : friendResponseList ) {
-
-                    executor.submit(new FacebookFriendFeedTask(this, friend.getId()));
-                }
-                friendPaging = friendResponseList.getPaging();
-                friendResponseList = client.fetchNext(friendPaging);
-            } while( friendPaging != null &&
-                    friendResponseList != null );
-        } catch (FacebookException e) {
-            e.printStackTrace();
-        }
-
-    }
-
-    protected Facebook getFacebookClient()
-    {
-        ConfigurationBuilder cb = new ConfigurationBuilder();
-        cb.setDebugEnabled(true)
-            .setOAuthAppId(configuration.getOauth().getAppId())
-            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
-            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
-            .setOAuthPermissions(ALL_PERMISSIONS)
-            .setJSONStoreEnabled(true)
-            .setClientVersion("v1.0");
-
-        FacebookFactory ff = new FacebookFactory(cb.build());
-        Facebook facebook = ff.getInstance();
-
-        return facebook;
-    }
-
-    @Override
-    public void cleanUp() {
-        shutdownAndAwaitTermination(executor);
-    }
-
-    private class FacebookFriendFeedTask implements Runnable {
-
-        FacebookFriendFeedProvider provider;
-        Facebook client;
-        String id;
-
-        public FacebookFriendFeedTask(FacebookFriendFeedProvider provider, String id) {
-            this.provider = provider;
-            this.id = id;
-        }
-
-        @Override
-        public void run() {
-            client = provider.getFacebookClient();
-                try {
-                    ResponseList<Post> postResponseList = client.getFeed(id);
-                    Paging<Post> postPaging;
-                    do {
-
-                        for (Post item : postResponseList) {
-                            String json = DataObjectFactory.getRawJSON(item);
-                            org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
-                            try {
-                                lock.readLock().lock();
-                                ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
-                                countersCurrent.incrementAttempt();
-                            } finally {
-                                lock.readLock().unlock();
-                            }
-                        }
-                        postPaging = postResponseList.getPaging();
-                        postResponseList = client.fetchNext(postPaging);
-                    } while( postPaging != null &&
-                            postResponseList != null );
-
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
deleted file mode 100644
index a111823..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookFriendUpdatesProvider.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.provider;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigRenderOptions;
-import facebook4j.*;
-import facebook4j.conf.ConfigurationBuilder;
-import facebook4j.json.DataObjectFactory;
-import org.apache.streams.config.StreamsConfigurator;
-import org.apache.streams.core.DatumStatusCounter;
-import org.apache.streams.core.StreamsDatum;
-import org.apache.streams.core.StreamsProvider;
-import org.apache.streams.core.StreamsResultSet;
-import org.apache.streams.facebook.FacebookUserInformationConfiguration;
-import org.apache.streams.facebook.FacebookUserstreamConfiguration;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.util.ComponentUtils;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import sun.reflect.generics.reflectiveObjects.NotImplementedException;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigInteger;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class FacebookFriendUpdatesProvider implements StreamsProvider, Serializable
-{
-
-    public static final String STREAMS_ID = "FacebookFriendPostsProvider";
-    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookFriendUpdatesProvider.class);
-
-    private static final ObjectMapper mapper = new StreamsJacksonMapper();
-
-    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
-    private FacebookUserstreamConfiguration configuration;
-
-    private Class klass;
-    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
-
-    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
-
-    public FacebookUserstreamConfiguration getConfig()              { return configuration; }
-
-    public void setConfig(FacebookUserstreamConfiguration config)   { this.configuration = config; }
-
-    protected Iterator<String[]> idsBatches;
-
-    protected ExecutorService executor;
-
-    protected DateTime start;
-    protected DateTime end;
-
-    protected final AtomicBoolean running = new AtomicBoolean();
-
-    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
-    private DatumStatusCounter countersTotal = new DatumStatusCounter();
-
-    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
-        return new ThreadPoolExecutor(nThreads, nThreads,
-                5000L, TimeUnit.MILLISECONDS,
-                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
-    }
-
-    public FacebookFriendUpdatesProvider() {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration configuration;
-        try {
-            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-    }
-
-    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config) {
-        this.configuration = config;
-    }
-
-    public FacebookFriendUpdatesProvider(Class klass) {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration configuration;
-        try {
-            configuration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-        this.klass = klass;
-    }
-
-    public FacebookFriendUpdatesProvider(FacebookUserstreamConfiguration config, Class klass) {
-        this.configuration = config;
-        this.klass = klass;
-    }
-
-    public Queue<StreamsDatum> getProviderQueue() {
-        return this.providerQueue;
-    }
-
-    @Override
-    public void startStream() {
-        running.set(true);
-    }
-
-    public StreamsResultSet readCurrent() {
-
-        Preconditions.checkArgument(idsBatches.hasNext());
-
-        LOGGER.info("readCurrent");
-
-        // return stuff
-
-        LOGGER.info("Finished.  Cleaning up...");
-
-        LOGGER.info("Providing {} docs", providerQueue.size());
-
-        StreamsResultSet result =  new StreamsResultSet(providerQueue);
-        running.set(false);
-
-        LOGGER.info("Exiting");
-
-        return result;
-
-    }
-
-    public StreamsResultSet readNew(BigInteger sequence) {
-        LOGGER.debug("{} readNew", STREAMS_ID);
-        throw new NotImplementedException();
-    }
-
-    public StreamsResultSet readRange(DateTime start, DateTime end) {
-        LOGGER.debug("{} readRange", STREAMS_ID);
-        this.start = start;
-        this.end = end;
-        readCurrent();
-        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
-        return result;
-    }
-
-    @Override
-    public boolean isRunning() {
-        return running.get();
-    }
-
-    void shutdownAndAwaitTermination(ExecutorService pool) {
-        pool.shutdown(); // Disable new tasks from being submitted
-        try {
-            // Wait a while for existing tasks to terminate
-            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
-                pool.shutdownNow(); // Cancel currently executing tasks
-                // Wait a while for tasks to respond to being cancelled
-                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
-                    System.err.println("Pool did not terminate");
-            }
-        } catch (InterruptedException ie) {
-            // (Re-)Cancel if current thread also interrupted
-            pool.shutdownNow();
-            // Preserve interrupt status
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public void prepare(Object o) {
-
-        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
-
-        Preconditions.checkNotNull(providerQueue);
-        Preconditions.checkNotNull(this.klass);
-        Preconditions.checkNotNull(configuration.getOauth().getAppId());
-        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
-        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
-
-        Facebook client = getFacebookClient();
-
-        try {
-            ResponseList<Friend> friendResponseList = client.friends().getFriends();
-            Paging<Friend> friendPaging;
-            do {
-
-                for( Friend friend : friendResponseList ) {
-
-                    //client.rawAPI().callPostAPI();
-                    // add a subscription
-                }
-                friendPaging = friendResponseList.getPaging();
-                friendResponseList = client.fetchNext(friendPaging);
-            } while( friendPaging != null &&
-                    friendResponseList != null );
-        } catch (FacebookException e) {
-            e.printStackTrace();
-        }
-
-    }
-
-    protected Facebook getFacebookClient()
-    {
-        ConfigurationBuilder cb = new ConfigurationBuilder();
-        cb.setDebugEnabled(true)
-            .setOAuthAppId(configuration.getOauth().getAppId())
-            .setOAuthAppSecret(configuration.getOauth().getAppSecret())
-            .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
-            .setOAuthPermissions(ALL_PERMISSIONS)
-            .setJSONStoreEnabled(true)
-            .setClientVersion("v1.0");
-
-        FacebookFactory ff = new FacebookFactory(cb.build());
-        Facebook facebook = ff.getInstance();
-
-        return facebook;
-    }
-
-    @Override
-    public void cleanUp() {
-        shutdownAndAwaitTermination(executor);
-    }
-
-    private class FacebookFeedPollingTask implements Runnable {
-
-        FacebookUserstreamProvider provider;
-        Facebook client;
-
-        private Set<Post> priorPollResult = Sets.newHashSet();
-
-        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
-            provider = facebookUserstreamProvider;
-        }
-
-        @Override
-        public void run() {
-            client = provider.getFacebookClient();
-            while (provider.isRunning()) {
-                try {
-                    ResponseList<Post> postResponseList = client.getHome();
-                    Set<Post> update = Sets.newHashSet(postResponseList);
-                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
-                    Set<Post> entrySet = Sets.difference(update, repeats);
-                    for (Post item : entrySet) {
-                        String json = DataObjectFactory.getRawJSON(item);
-                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
-                        try {
-                            lock.readLock().lock();
-                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
-                            countersCurrent.incrementAttempt();
-                        } finally {
-                            lock.readLock().unlock();
-                        }
-                    }
-                    priorPollResult = update;
-                    Thread.sleep(configuration.getPollIntervalMillis());
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
deleted file mode 100644
index 8640f5d..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserInformationProvider.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.provider;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigRenderOptions;
-import facebook4j.*;
-import facebook4j.conf.ConfigurationBuilder;
-import facebook4j.json.DataObjectFactory;
-import org.apache.streams.config.StreamsConfigurator;
-import org.apache.streams.core.StreamsDatum;
-import org.apache.streams.core.StreamsProvider;
-import org.apache.streams.core.StreamsResultSet;
-import org.apache.streams.facebook.FacebookUserInformationConfiguration;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import sun.reflect.generics.reflectiveObjects.NotImplementedException;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigInteger;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class FacebookUserInformationProvider implements StreamsProvider, Serializable
-{
-
-    public static final String STREAMS_ID = "FacebookUserInformationProvider";
-    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserInformationProvider.class);
-
-    private static final ObjectMapper mapper = new StreamsJacksonMapper();
-
-    private static final String ALL_PERMISSIONS = "ads_management,ads_read,create_event,create_note,email,export_stream,friends_about_me,friends_actions.books,friends_actions.music,friends_actions.news,friends_actions.video,friends_activities,friends_birthday,friends_education_history,friends_events,friends_games_activity,friends_groups,friends_hometown,friends_interests,friends_likes,friends_location,friends_notes,friends_online_presence,friends_photo_video_tags,friends_photos,friends_questions,friends_relationship_details,friends_relationships,friends_religion_politics,friends_status,friends_subscriptions,friends_videos,friends_website,friends_work_history,manage_friendlists,manage_notifications,manage_pages,photo_upload,publish_actions,publish_stream,read_friendlists,read_insights,read_mailbox,read_page_mailboxes,read_requests,read_stream,rsvp_event,share_item,sms,status_update,user_about_me,user_actions.books,user_actions.music,user_actions.news,user_actions.video,user_activitie
 s,user_birthday,user_education_history,user_events,user_friends,user_games_activity,user_groups,user_hometown,user_interests,user_likes,user_location,user_notes,user_online_presence,user_photo_video_tags,user_photos,user_questions,user_relationship_details,user_relationships,user_religion_politics,user_status,user_subscriptions,user_videos,user_website,user_work_history,video_upload,xmpp_login";
-    private FacebookUserInformationConfiguration facebookUserInformationConfiguration;
-
-    private Class klass;
-    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
-
-    public FacebookUserInformationConfiguration getConfig()              { return facebookUserInformationConfiguration; }
-
-    public void setConfig(FacebookUserInformationConfiguration config)   { this.facebookUserInformationConfiguration = config; }
-
-    protected Iterator<String[]> idsBatches;
-
-    protected ExecutorService executor;
-
-    protected DateTime start;
-    protected DateTime end;
-
-    protected final AtomicBoolean running = new AtomicBoolean();
-
-    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
-        return new ThreadPoolExecutor(nThreads, nThreads,
-                5000L, TimeUnit.MILLISECONDS,
-                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
-    }
-
-    public FacebookUserInformationProvider() {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
-        try {
-            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-    }
-
-    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config) {
-        this.facebookUserInformationConfiguration = config;
-    }
-
-    public FacebookUserInformationProvider(Class klass) {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
-        try {
-            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-        this.klass = klass;
-    }
-
-    public FacebookUserInformationProvider(FacebookUserInformationConfiguration config, Class klass) {
-        this.facebookUserInformationConfiguration = config;
-        this.klass = klass;
-    }
-
-    public Queue<StreamsDatum> getProviderQueue() {
-        return this.providerQueue;
-    }
-
-    @Override
-    public void startStream() {
-        running.set(true);
-    }
-
-    public StreamsResultSet readCurrent() {
-
-        Preconditions.checkArgument(idsBatches.hasNext());
-
-        LOGGER.info("readCurrent");
-
-        Facebook client = getFacebookClient();
-
-        try {
-            User me = client.users().getMe();
-            String json = mapper.writeValueAsString(me);
-            providerQueue.add(
-                new StreamsDatum(json, DateTime.now())
-            );
-        } catch (JsonProcessingException e) {
-            e.printStackTrace();
-        } catch (FacebookException e) {
-            e.printStackTrace();
-        }
-
-        if( idsBatches.hasNext()) {
-            while (idsBatches.hasNext()) {
-                try {
-                    List<User> userList = client.users().getUsers(idsBatches.next());
-                    for (User user : userList) {
-
-                        try {
-                            String json = mapper.writeValueAsString(user);
-                            providerQueue.add(
-                                    new StreamsDatum(json, DateTime.now())
-                            );
-                        } catch (JsonProcessingException e) {
-                            //                        e.printStackTrace();
-                        }
-                    }
-
-                } catch (FacebookException e) {
-                    e.printStackTrace();
-                }
-            }
-        } else {
-            try {
-                ResponseList<Friend> friendResponseList = client.friends().getFriends();
-                Paging<Friend> friendPaging;
-                do {
-
-                    for( Friend friend : friendResponseList ) {
-
-                        String json;
-                        try {
-                            json = mapper.writeValueAsString(friend);
-                            providerQueue.add(
-                                    new StreamsDatum(json)
-                            );
-                        } catch (JsonProcessingException e) {
-//                        e.printStackTrace();
-                        }
-                    }
-                    friendPaging = friendResponseList.getPaging();
-                    friendResponseList = client.fetchNext(friendPaging);
-                } while( friendPaging != null &&
-                        friendResponseList != null );
-            } catch (FacebookException e) {
-                e.printStackTrace();
-            }
-
-        }
-
-        LOGGER.info("Finished.  Cleaning up...");
-
-        LOGGER.info("Providing {} docs", providerQueue.size());
-
-        StreamsResultSet result =  new StreamsResultSet(providerQueue);
-        running.set(false);
-
-        LOGGER.info("Exiting");
-
-        return result;
-
-    }
-
-    public StreamsResultSet readNew(BigInteger sequence) {
-        LOGGER.debug("{} readNew", STREAMS_ID);
-        throw new NotImplementedException();
-    }
-
-    public StreamsResultSet readRange(DateTime start, DateTime end) {
-        LOGGER.debug("{} readRange", STREAMS_ID);
-        this.start = start;
-        this.end = end;
-        readCurrent();
-        StreamsResultSet result = (StreamsResultSet)providerQueue.iterator();
-        return result;
-    }
-
-    @Override
-    public boolean isRunning() {
-        return running.get();
-    }
-
-    void shutdownAndAwaitTermination(ExecutorService pool) {
-        pool.shutdown(); // Disable new tasks from being submitted
-        try {
-            // Wait a while for existing tasks to terminate
-            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
-                pool.shutdownNow(); // Cancel currently executing tasks
-                // Wait a while for tasks to respond to being cancelled
-                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
-                    System.err.println("Pool did not terminate");
-            }
-        } catch (InterruptedException ie) {
-            // (Re-)Cancel if current thread also interrupted
-            pool.shutdownNow();
-            // Preserve interrupt status
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public void prepare(Object o) {
-
-        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
-
-        Preconditions.checkNotNull(providerQueue);
-        Preconditions.checkNotNull(this.klass);
-        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppId());
-        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getAppSecret());
-        Preconditions.checkNotNull(facebookUserInformationConfiguration.getOauth().getUserAccessToken());
-        Preconditions.checkNotNull(facebookUserInformationConfiguration.getInfo());
-
-        List<String> ids = new ArrayList<String>();
-        List<String[]> idsBatches = new ArrayList<String[]>();
-
-        for(String s : facebookUserInformationConfiguration.getInfo()) {
-            if(s != null)
-            {
-                ids.add(s);
-
-                if(ids.size() >= 100) {
-                    // add the batch
-                    idsBatches.add(ids.toArray(new String[ids.size()]));
-                    // reset the Ids
-                    ids = new ArrayList<String>();
-                }
-
-            }
-        }
-
-        if(ids.size() > 0)
-            idsBatches.add(ids.toArray(new String[ids.size()]));
-
-        this.idsBatches = idsBatches.iterator();
-    }
-
-    protected Facebook getFacebookClient()
-    {
-        ConfigurationBuilder cb = new ConfigurationBuilder();
-        cb.setDebugEnabled(true)
-            .setOAuthAppId(facebookUserInformationConfiguration.getOauth().getAppId())
-            .setOAuthAppSecret(facebookUserInformationConfiguration.getOauth().getAppSecret())
-            .setOAuthAccessToken(facebookUserInformationConfiguration.getOauth().getUserAccessToken())
-            .setOAuthPermissions(ALL_PERMISSIONS)
-            .setJSONStoreEnabled(true)
-            .setClientVersion("v1.0");
-
-        FacebookFactory ff = new FacebookFactory(cb.build());
-        Facebook facebook = ff.getInstance();
-
-        return facebook;
-    }
-
-    @Override
-    public void cleanUp() {
-        shutdownAndAwaitTermination(executor);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java b/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
deleted file mode 100644
index b0bf082..0000000
--- a/streams-contrib/streams-provider-facebook/src/main/java/com/facebook/provider/FacebookUserstreamProvider.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *
- *   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 com.facebook.provider;
-
-import com.facebook.*;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigRenderOptions;
-import facebook4j.*;
-import facebook4j.Post;
-import facebook4j.conf.ConfigurationBuilder;
-import facebook4j.json.DataObjectFactory;
-import org.apache.streams.config.StreamsConfigurator;
-import org.apache.streams.core.DatumStatusCounter;
-import org.apache.streams.core.StreamsDatum;
-import org.apache.streams.core.StreamsProvider;
-import org.apache.streams.core.StreamsResultSet;
-import org.apache.streams.facebook.FacebookUserInformationConfiguration;
-import org.apache.streams.facebook.FacebookUserstreamConfiguration;
-import org.apache.streams.jackson.StreamsJacksonMapper;
-import org.apache.streams.util.ComponentUtils;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import sun.reflect.generics.reflectiveObjects.NotImplementedException;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigInteger;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class FacebookUserstreamProvider implements StreamsProvider, Serializable {
-
-    public static final String STREAMS_ID = "FacebookUserstreamProvider";
-    private static final Logger LOGGER = LoggerFactory.getLogger(FacebookUserstreamProvider.class);
-
-    private static final ObjectMapper mapper = new StreamsJacksonMapper();
-
-    private static final String ALL_PERMISSIONS = "read_stream";
-    private FacebookUserstreamConfiguration configuration;
-
-    private Class klass;
-    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
-
-    protected volatile Queue<StreamsDatum> providerQueue = new LinkedBlockingQueue<StreamsDatum>();
-
-    public FacebookUserstreamConfiguration getConfig() {
-        return configuration;
-    }
-
-    public void setConfig(FacebookUserstreamConfiguration config) {
-        this.configuration = config;
-    }
-
-    protected ListeningExecutorService executor;
-
-    protected DateTime start;
-    protected DateTime end;
-
-    protected final AtomicBoolean running = new AtomicBoolean();
-
-    private DatumStatusCounter countersCurrent = new DatumStatusCounter();
-    private DatumStatusCounter countersTotal = new DatumStatusCounter();
-
-    protected Facebook client;
-
-    private static ExecutorService newFixedThreadPoolWithQueueSize(int nThreads, int queueSize) {
-        return new ThreadPoolExecutor(nThreads, nThreads,
-                5000L, TimeUnit.MILLISECONDS,
-                new ArrayBlockingQueue<Runnable>(queueSize, true), new ThreadPoolExecutor.CallerRunsPolicy());
-    }
-
-    public FacebookUserstreamProvider() {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
-        try {
-            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-    }
-
-    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config) {
-        this.configuration = config;
-    }
-
-    public FacebookUserstreamProvider(Class klass) {
-        Config config = StreamsConfigurator.config.getConfig("facebook");
-        FacebookUserInformationConfiguration facebookUserInformationConfiguration;
-        try {
-            facebookUserInformationConfiguration = mapper.readValue(config.root().render(ConfigRenderOptions.concise()), FacebookUserInformationConfiguration.class);
-        } catch (IOException e) {
-            e.printStackTrace();
-            return;
-        }
-        this.klass = klass;
-    }
-
-    public FacebookUserstreamProvider(FacebookUserstreamConfiguration config, Class klass) {
-        this.configuration = config;
-        this.klass = klass;
-    }
-
-    public Queue<StreamsDatum> getProviderQueue() {
-        return this.providerQueue;
-    }
-
-    @Override
-    public void startStream() {
-
-        client = getFacebookClient();
-
-        if( configuration.getInfo() != null &&
-            configuration.getInfo().size() > 0 ) {
-            for( String id : configuration.getInfo()) {
-                executor.submit(new FacebookFeedPollingTask(this, id));
-            }
-            running.set(true);
-        } else {
-            try {
-                String id = client.getMe().getId();
-                executor.submit(new FacebookFeedPollingTask(this, id));
-                running.set(true);
-            } catch (FacebookException e) {
-                LOGGER.error(e.getMessage());
-                running.set(false);
-            }
-        }
-    }
-
-    public StreamsResultSet readCurrent() {
-
-        StreamsResultSet current;
-
-        synchronized (FacebookUserstreamProvider.class) {
-            current = new StreamsResultSet(Queues.newConcurrentLinkedQueue(providerQueue));
-            current.setCounter(new DatumStatusCounter());
-            current.getCounter().add(countersCurrent);
-            countersTotal.add(countersCurrent);
-            countersCurrent = new DatumStatusCounter();
-            providerQueue.clear();
-        }
-
-        return current;
-
-    }
-
-    public StreamsResultSet readNew(BigInteger sequence) {
-        LOGGER.debug("{} readNew", STREAMS_ID);
-        throw new NotImplementedException();
-    }
-
-    public StreamsResultSet readRange(DateTime start, DateTime end) {
-        LOGGER.debug("{} readRange", STREAMS_ID);
-        this.start = start;
-        this.end = end;
-        readCurrent();
-        StreamsResultSet result = (StreamsResultSet) providerQueue.iterator();
-        return result;
-    }
-
-    @Override
-    public boolean isRunning() {
-        return running.get();
-    }
-
-    void shutdownAndAwaitTermination(ExecutorService pool) {
-        pool.shutdown(); // Disable new tasks from being submitted
-        try {
-            // Wait a while for existing tasks to terminate
-            if (!pool.awaitTermination(10, TimeUnit.SECONDS)) {
-                pool.shutdownNow(); // Cancel currently executing tasks
-                // Wait a while for tasks to respond to being cancelled
-                if (!pool.awaitTermination(10, TimeUnit.SECONDS))
-                    System.err.println("Pool did not terminate");
-            }
-        } catch (InterruptedException ie) {
-            // (Re-)Cancel if current thread also interrupted
-            pool.shutdownNow();
-            // Preserve interrupt status
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public void prepare(Object o) {
-
-        executor = MoreExecutors.listeningDecorator(newFixedThreadPoolWithQueueSize(5, 20));
-
-        Preconditions.checkNotNull(providerQueue);
-        Preconditions.checkNotNull(this.klass);
-        Preconditions.checkNotNull(configuration.getOauth().getAppId());
-        Preconditions.checkNotNull(configuration.getOauth().getAppSecret());
-        Preconditions.checkNotNull(configuration.getOauth().getUserAccessToken());
-
-        client = getFacebookClient();
-
-        if( configuration.getInfo() != null &&
-            configuration.getInfo().size() > 0 ) {
-
-            List<String> ids = new ArrayList<String>();
-            List<String[]> idsBatches = new ArrayList<String[]>();
-
-            for (String s : configuration.getInfo()) {
-                if (s != null) {
-                    ids.add(s);
-
-                    if (ids.size() >= 100) {
-                        // add the batch
-                        idsBatches.add(ids.toArray(new String[ids.size()]));
-                        // reset the Ids
-                        ids = new ArrayList<String>();
-                    }
-
-                }
-            }
-        }
-    }
-
-    protected Facebook getFacebookClient() {
-        ConfigurationBuilder cb = new ConfigurationBuilder();
-        cb.setDebugEnabled(true)
-                .setOAuthAppId(configuration.getOauth().getAppId())
-                .setOAuthAppSecret(configuration.getOauth().getAppSecret())
-                .setOAuthAccessToken(configuration.getOauth().getUserAccessToken())
-                .setOAuthPermissions(ALL_PERMISSIONS)
-                .setJSONStoreEnabled(true);
-
-        FacebookFactory ff = new FacebookFactory(cb.build());
-        Facebook facebook = ff.getInstance();
-
-        return facebook;
-    }
-
-    @Override
-    public void cleanUp() {
-        shutdownAndAwaitTermination(executor);
-    }
-
-    private class FacebookFeedPollingTask implements Runnable {
-
-        FacebookUserstreamProvider provider;
-        Facebook client;
-        String id;
-
-        private Set<Post> priorPollResult = Sets.newHashSet();
-
-        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider) {
-            this.provider = facebookUserstreamProvider;
-        }
-
-        public FacebookFeedPollingTask(FacebookUserstreamProvider facebookUserstreamProvider, String id) {
-            this.provider = facebookUserstreamProvider;
-            this.client = provider.client;
-            this.id = id;
-        }
-        @Override
-        public void run() {
-            while (provider.isRunning()) {
-                ResponseList<Post> postResponseList;
-                try {
-                    postResponseList = client.getFeed(id);
-
-                    Set<Post> update = Sets.newHashSet(postResponseList);
-                    Set<Post> repeats = Sets.intersection(priorPollResult, Sets.newHashSet(update));
-                    Set<Post> entrySet = Sets.difference(update, repeats);
-                    LOGGER.debug(this.id + " response: " + update.size() + " previous: " + repeats.size() + " new: " + entrySet.size());
-                    for (Post item : entrySet) {
-                        String json = DataObjectFactory.getRawJSON(item);
-                        org.apache.streams.facebook.Post post = mapper.readValue(json, org.apache.streams.facebook.Post.class);
-                        try {
-                            lock.readLock().lock();
-                            ComponentUtils.offerUntilSuccess(new StreamsDatum(post), providerQueue);
-                            countersCurrent.incrementAttempt();
-                        } finally {
-                            lock.readLock().unlock();
-                        }
-                    }
-                    priorPollResult = update;
-                } catch (Exception e) {
-                    e.printStackTrace();
-                } finally {
-                    try {
-                        Thread.sleep(configuration.getPollIntervalMillis());
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                }
-            }
-        }
-    }
-}


[10/15] updated packages

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/dc432af2/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDC.xml
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDC.xml b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDC.xml
index 34d4db5..1dbf140 100644
--- a/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDC.xml
+++ b/streams-contrib/streams-provider-gnip/gnip-edc-reddit/src/test/resources/RedditEDC.xml
@@ -8,7 +8,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/longrange/comments/1l7fgd/need_advice_selecting_between_two_rifles/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_135v09"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_135v09"/>
 			<title>Reddit - Keyword - Search (Recent) - "Remington"</title>
 			<updated>2013-08-28T19-54-43Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Remington"
@@ -56,7 +56,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/KerbalSpaceProgram/comments/1l8rv5/tip_choosing_a_jet_engine_based_on_velocity/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_19h0y5"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_19h0y5"/>
 			<title>Reddit - Keyword - Search (Recent) - "Max Factor"</title>
 			<updated>2013-08-28T19-54-00Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Max Factor"
@@ -165,7 +165,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/electronic_cigarette/comments/1l7v1p/first_mech_ordered_what_am_i_missing/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>
 			<title>Reddit - Keyword - Search (Recent) - "Panasonic"</title>
 			<updated>2013-08-28T19-47-07Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Panasonic"
@@ -227,7 +227,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/videography/comments/1l9kpj/advice_dslr_vs_camcorders_for_wedding_videography/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Panasonic%22&amp;limit=100&amp;after=t3_168ejl"/>
 			<title>Reddit - Keyword - Search (Recent) - "Panasonic"</title>
 			<updated>2013-08-28T19-47-07Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Panasonic"
@@ -287,7 +287,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/RandomActsofMakeup/comments/1l7xtm/contest_im_hungry_go_make_me_something/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_sa7ex"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_sa7ex"/>
 			<title>Reddit - Keyword - Search (Recent) - "L'oreal" "loreal"</title>
 			<updated>2013-08-28T19-46-28Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"L'oreal" "loreal"
@@ -367,7 +367,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/headphones/comments/1l8bju/recommendations_for_a_recreational_gaming_pc/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_156seg"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Max%20Factor%22&amp;limit=100&amp;after=t3_156seg"/>
 			<title>Reddit - Keyword - Search (Recent) - "Max Factor"</title>
 			<updated>2013-08-28T19-39-36Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Max Factor"
@@ -410,7 +410,7 @@
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SanDiegoClassifieds/comments/1l7vbs/anyone_need_a_large_hepa_filter_ive_got_a_two/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100"/>
 			<title>Reddit - Keyword - Search (Recent) - "lysol"</title>
 			<updated>2013-08-28T19-35-40Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"lysol"
@@ -458,7 +458,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Fapucational/comments/1l9ek9/by_1890_many_variations_of_soap_were_offered_with/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22colgate%22&amp;limit=100&amp;after=t3_1cfq0r"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22colgate%22&amp;limit=100&amp;after=t3_1cfq0r"/>
 			<title>Reddit - Keyword - Search (Recent) - "colgate"</title>
 			<updated>2013-08-28T19-34-16Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"colgate"
@@ -501,7 +501,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l9u33/been_sick_for_days_but_gifts_from_loreal_perked/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_1a5xa8"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22L%27oreal%22%20%22loreal%22&amp;limit=100&amp;after=t3_1a5xa8"/>
 			<title>Reddit - Keyword - Search (Recent) - "L'oreal" "loreal"</title>
 			<updated>2013-08-28T19-32-14Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"L'oreal" "loreal"
@@ -544,7 +544,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/amazonsdeal/comments/1l9g7d/amazon_lysol_disinfecting_wipes_upto_35_off_plus/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100&amp;after=t3_10zvhy"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22lysol%22&amp;limit=100&amp;after=t3_10zvhy"/>
 			<title>Reddit - Keyword - Search (Recent) - "lysol"</title>
 			<updated>2013-08-28T19-28-30Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"lysol"
@@ -587,7 +587,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shotguns/comments/1l7ym9/remington_1100_stock/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_1fk3mo"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Remington%22&amp;limit=100&amp;after=t3_1fk3mo"/>
 			<title>Reddit - Keyword - Search (Recent) - "Remington"</title>
 			<updated>2013-08-28T19-25-58Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Remington"
@@ -631,7 +631,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/SkincareAddiction/comments/1la0ec/need_help_finding_a_new_skincare_routine/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22Aveeno%22&amp;limit=100"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22Aveeno%22&amp;limit=100"/>
 			<title>Reddit - Keyword - Search (Recent) - "Aveeno"</title>
 			<updated>2013-08-28T19-24-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"Aveeno"
@@ -689,7 +689,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Shave_Bazaar/comments/1l85hv/fs_den_clearance_fatboy_rocket_simpson/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22schick%22&amp;limit=100&amp;after=t3_13oj1g"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22schick%22&amp;limit=100&amp;after=t3_13oj1g"/>
 			<title>Reddit - Keyword - Search (Recent) - "schick"</title>
 			<updated>2013-08-28T19-22-35Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"schick"
@@ -897,7 +897,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MakeupAddiction/comments/1l7tb6/need_help_any_opinions_about_loreal_cosmetics/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22L%27oreal%22%20%22loreal%22%20%29&amp;limit=100&amp;after=t3_107krl"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%28%22L%27oreal%22%20%22loreal%22%20%29&amp;limit=100&amp;after=t3_107krl"/>
 			<title>Reddit - Keyword - Search (Recent) - ("L'oreal" "loreal" )</title>
 			<updated>2013-08-28T18-40-15Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("L'oreal" "loreal" )
@@ -945,7 +945,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EarthPorn/comments/1l7btz/cutthroat_pass_pacific_crest_trail_wa_2560x1920/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22crest%22%20%29%20-%20%28%22pacific%20crest%22%20%29&amp;limit=100&amp;after=t3_10m52h"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%28%22crest%22%20%29%20-%20%28%22pacific%20crest%22%20%29&amp;limit=100&amp;after=t3_10m52h"/>
 			<title>Reddit - Keyword - Search (Recent) - ("crest" ) - ("pacific crest" )</title>
 			<updated>2013-08-28T18-03-31Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("crest" ) - ("pacific crest" )
@@ -988,7 +988,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MLPLounge/comments/1l842y/colgate_beadsprite_courtesy_of_opti/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22colgate%22%20%29&amp;limit=100&amp;after=t3_j7xv0"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%28%22colgate%22%20%29&amp;limit=100&amp;after=t3_j7xv0"/>
 			<title>Reddit - Keyword - Search (Recent) - ("colgate" )</title>
 			<updated>2013-08-28T18-03-27Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("colgate" )
@@ -1030,7 +1030,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EQNext/comments/1l8qmr/raids_without_loot/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%28%22Max%20Factor%22%20%29&amp;limit=100&amp;after=t3_zkcy0"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%28%22Max%20Factor%22%20%29&amp;limit=100&amp;after=t3_zkcy0"/>
 			<title>Reddit - Keyword - Search (Recent) - ("Max Factor" )</title>
 			<updated>2013-08-28T18-01-30Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">("Max Factor" )
@@ -1090,7 +1090,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/realasians/comments/1brddb/i_guess_ill_go_outside_after_this_sunshine_ftw_d/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1133,7 +1133,7 @@ it&amp;amp;#39;s used and will have to be hit with lysol but at this price... it
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/trees/comments/i5fxh/chronic_cheesecake_recipe/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1202,7 +1202,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/todayilearned/comments/y8nml/til_that_the_average_lemon_has_2_oz_of_lemon/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1245,7 +1245,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/wicked_edge/comments/1ax1ok/anyone_ever_try_lathering_up_a_soap_not/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1289,7 +1289,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/1c8oyd/potato_and_yam_fries_with_parsley_and_lemon_zest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1332,7 +1332,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1k9npy/2013_wcg_korea_national_final_parting_vs_rain/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1375,7 +1375,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/seduction/comments/1gx92i/billys_guide_to_demonstrating_high_value/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1476,7 +1476,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/tonightsdinner/comments/1drjw4/ribeye_artichoke_gratinata_linguine_with_garlic/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1519,7 +1519,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/realasians/comments/1bk1u0/not_a_cookie_cutter_cutting_cookies_3/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1562,7 +1562,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/MLPLounge/comments/13gakl/he_is_quite_possibly_the_most_boring_man_in_the/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1615,7 +1615,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Paleo/comments/18r1gq/fantastic_noegg_quick_simple_onthego_paleo/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1668,7 +1668,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/1hyj0o/smoked_salmon_on_buttered_sour_dough_bread_topped/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1711,7 +1711,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/bourbon/comments/qndjo/the_best_bourbon_for_an_old_fashioned/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1761,7 +1761,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/exmormon/comments/crybg/best_quote_ever/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1808,7 +1808,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/WTF/comments/70cjh/3000_of_audiophile_stupidity_cryogenically/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1850,7 +1850,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskCulinary/comments/13ikau/aromatic_bitters_in_apple_pie/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1895,7 +1895,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/NoFap/comments/1dxl50/life_is_great/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1938,7 +1938,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/depression/comments/z9z8v/kanye_jest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -1992,7 +1992,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/food/comments/xv05f/margarita_cupcakes_brushed_with_tequila_topped/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2035,7 +2035,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AdviceAnimals/comments/14ngif/happened_to_my_brother_i_tried_to_warn_him/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2078,7 +2078,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/hookah/comments/16dl6a/blueberry_muffin_mandarin_zest_lemon_mint_and_a/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2121,7 +2121,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/cocktails/comments/14viax/dark_as_night_homemade_bitters_with_black_tea/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2164,7 +2164,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/cocktails/comments/19uyp9/dehydrating_citrus_zest_for_cocktail_presentation/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2207,7 +2207,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/fffffffuuuuuuuuuuuu/comments/z16we/lemon_zest_glade_rage/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2250,7 +2250,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/canada/comments/g8yqj/canadians_dont_share_harpers_zest_for_fighter/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2292,7 +2292,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Homebrewing/comments/17sfsk/adding_orange_zest_to_a_beer_do_i_need_to_worry/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2336,7 +2336,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskReddit/comments/x4qnn/what_is_your_favorite_weird_food_combination/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2379,7 +2379,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/relationships/comments/1c7elk/i_26m_love_my_girlfriend_f23_but_for_our/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2436,7 +2436,7 @@ Place cream cheese and sour cream in a large bowl. Mix for 30 seconds. Add whole
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/buildapc/comments/191wco/guide_level_beginner_to_enthusiasts_ravioli_from/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2583,7 +2583,7 @@ And I think many of you know Siracha sauce. Adds hot and spicy to your food, wor
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Homebrewing/comments/18kezw/as_one_of_my_favorite_styles_i_finally_made_my/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2626,7 +2626,7 @@ And I think many of you know Siracha sauce. Adds hot and spicy to your food, wor
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/1d7nxv/lime_cheesecake_with_mango_ginger_sauce_and/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2669,7 +2669,7 @@ And I think many of you know Siracha sauce. Adds hot and spicy to your food, wor
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1eoxnb/spl_round_5_week_3_team_8_vs_kt_rolster_results/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2807,7 +2807,7 @@ And I think many of you know Siracha sauce. Adds hot and spicy to your food, wor
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1bwrfl/kt_rolster_vs_stx_soul_proleague_round_4_week_1/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2924,7 +2924,7 @@ And I think many of you know Siracha sauce. Adds hot and spicy to your food, wor
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Paleo/comments/18yd5u/paleo_orange_chicken/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -2988,7 +2988,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/52weeksofcooking/comments/12nldg/week_44_cinnamon_cinnamon_roll_taco_shell_filled/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3031,7 +3031,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/shittyfoodporn/comments/1d4w1x/spur_of_the_moment_from_scratch_vanilla_cake_with/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3074,7 +3074,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/shittyadvice/comments/kavxc/how_do_i_regain_my_creativity_and_zest_for_life/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3115,7 +3115,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/keto/comments/o6vei/here_it_is_the_unofficial_tell_me_what_to_eat/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3266,7 +3266,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/RedditLaqueristas/comments/lc3wt/zombie_zest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3309,7 +3309,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/loseit/comments/rgk2p/i_took_a_big_step_last_night/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3354,7 +3354,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Homebrewing/comments/vpv38/planning_a_saison_when_should_i_add/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3438,7 +3438,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/britishproblems/comments/1l7ahd/i_tried_a_carling_zest_and_quite_enjoyed_it/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3481,7 +3481,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Breadit/comments/147um0/cinnamontoasted_walnutcranberryorange_zest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3524,7 +3524,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1f4x65/spl_round_5_week_4_kt_vs_skt1_results/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3585,7 +3585,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/LifeProTips/comments/1g9j75/lpt_handy_uses_for_citrus_peels/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3629,7 +3629,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/cocktails/comments/ojkcy/cocktail_of_the_week_week_3_the_old_fashioned/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3691,7 +3691,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1i6s1j/hello_sc2_you_guys_seem_cool_greetings_from/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3765,7 +3765,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Cooking/comments/142bo4/what_to_do_with_citrus_rindzest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3808,7 +3808,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/depression/comments/1l3ws8/i_have_0_motivation_for_anything_no_interests_no/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3852,7 +3852,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/recipes/comments/obg42/what_are_you_having_for_dinner/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3908,7 +3908,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/recipes/comments/13rock/ive_been_experimenting_with_margarita_recipes/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -3972,7 +3972,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/realasians/comments/1bftaw/reaaaally_bored_this_morning/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4015,7 +4015,7 @@ zest from 1 orange
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/IAmA/comments/i3d2a/iama_22yearold_who_without_fail_every_year_on_the/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4090,7 +4090,7 @@ June 19, 1994: I was freeing the jail in Capture the Flag when I was blindsided
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskCulinary/comments/zzjbj/lemon_zest_for_mousse/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4133,7 +4133,7 @@ June 19, 1994: I was freeing the jail in Capture the Flag when I was blindsided
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/keto/comments/16zqc4/coconut_lime_ice_cream/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4187,7 +4187,7 @@ June 19, 1994: I was freeing the jail in Capture the Flag when I was blindsided
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/y4bxn/blueberry_and_jam_muffin_with_a_lemonzest_and/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4230,7 +4230,7 @@ June 19, 1994: I was freeing the jail in Capture the Flag when I was blindsided
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/eatsandwiches/comments/qgdic/capicola_and_muenster_on_cracked_wheat_sourdough/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4273,7 +4273,7 @@ June 19, 1994: I was freeing the jail in Capture the Flag when I was blindsided
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/EatCheapAndHealthy/comments/15unrx/make_your_own_cheap_healthy_syrups_for_your_soda/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4327,7 +4327,7 @@ We choose organic lemons and limes for this and use filtered poland spring water
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/legs/comments/1abhtv/zest_of_sleep/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4370,7 +4370,7 @@ We choose organic lemons and limes for this and use filtered poland spring water
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/recipes/comments/1f8eao/devil_sauce_aka_ambrosia_aka_the_last_sauce_you/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4481,7 +4481,7 @@ We choose organic lemons and limes for this and use filtered poland spring water
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/recipes/comments/16i6uq/guacamole_what_are_your_secrets/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4550,7 +4550,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/166zer/venison_tenderloin_with_turnip_puree_and_fried/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4593,7 +4593,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/recipes/comments/1aj9vs/coconut_bread_via_smittenkitchen_gonna_have_to/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4635,7 +4635,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Cooking/comments/1cop9n/lime_cordial/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4726,7 +4726,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskReddit/comments/hrp83/what_should_i_do_with_10_pounds_of_lemons/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4779,7 +4779,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Cooking/comments/1fz7cy/my_girlfriend_and_i_are_cooking_together_on/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4824,7 +4824,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/19dyai/cinnamon_oat_nutella_ice_cream_cola_cake_lime/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4867,7 +4867,7 @@ a little bit of sriracha
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/self/comments/130bnc/on_this_day_94_years_ago_the_first_world_war_came/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -4942,7 +4942,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1eooiv/spl_round_5_week_3_samsung_khan_vs_egliquid/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5065,7 +5065,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/FoodPorn/comments/12abmw/i_made_sealife_shaped_french_madeleines_with/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5108,7 +5108,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/trees/comments/o1nw3/if_i_put_orange_zest/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5152,7 +5152,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Frugal/comments/egw4n/small_tip_use_your_orange_peels/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5199,7 +5199,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/proper/comments/gicdc/i_say_this_is_the_distinguished_condiment_to_add/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5241,7 +5241,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/buildapc/comments/xzxf9/what_are_coolnifty_maybe_pointless_things_i_could/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5288,7 +5288,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1bqe7o/in_13_hours_proleague_returns_with_kt_flash_vs/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5352,7 +5352,7 @@ Pro patria mori.&amp;lt;/p&amp;gt;
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskReddit/comments/a3a1z/how_do_i_regain_my_reading_habit/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5397,7 +5397,7 @@ Has anyone experienced this and managed to regain their zest for reading? &amp;l
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/AskReddit/comments/koqsp/if_you_could_be_best_friends_with_any_celebrity/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5440,7 +5440,7 @@ Has anyone experienced this and managed to regain their zest for reading? &amp;l
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1dq1vm/spl_round_5_week_1_egliquid_vs_stx_soul_results/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5543,7 +5543,7 @@ Has anyone experienced this and managed to regain their zest for reading? &amp;l
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/starcraft/comments/1ds66p/spl_r5_w1_kt_rolster_41_samsung_khan/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5591,7 +5591,7 @@ Zest &amp;amp;lt; &amp;lt;strong&amp;gt;RorO&amp;lt;/strong&amp;gt;&amp;lt;br/&a
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/Homebrewing/comments/bwksb/hefeweizen_with_lemon_zest_thoughts_on_honey_in/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5639,7 +5639,7 @@ Zest &amp;amp;lt; &amp;lt;strong&amp;gt;RorO&amp;lt;/strong&amp;gt;&amp;lt;br/&a
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/tacos/comments/opy7e/a_peace_offering_from_banana_may_your_sauce_never/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5682,7 +5682,7 @@ Zest &amp;amp;lt; &amp;lt;strong&amp;gt;RorO&amp;lt;/strong&amp;gt;&amp;lt;br/&a
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/LV426/comments/1dmx9w/that_orange_sure_did_look_familiari_present_to/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""
@@ -5725,7 +5725,7 @@ Zest &amp;amp;lt; &amp;lt;strong&amp;gt;RorO&amp;lt;/strong&amp;gt;&amp;lt;br/&a
 		<category term="ArticlePosted" label="Article Posted"/>
 		<link rel="alternate" type="text/html" href="http://www.reddit.com/r/atheism/comments/1088dt/a_classy_new_way_to_respond_to_stupid_fundie/"/>
 		<source>
-			<link rel="self" type="application/json" href="http://com.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
+			<link rel="self" type="application/json" href="http://org.apache.streams.facebook.api.reddit.com/search.json?q=%22zest%22%20%20%22%22%20%20--%22%22&amp;limit=100&amp;after=t3_1brddb"/>
 			<title>Reddit - Keyword - Search (Recent) - "zest"  ""  --""</title>
 			<updated>2013-08-28T17-54-04Z</updated>
 			<gnip:rule xmlns:gnip="http://www.gnip.com/schemas/2010">"zest"  ""  --""