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

[1/9] git commit: Changes necessary to satisfy JIRA issues STREAMS-75, 76, 77 for Streams' URL Unwinder

Repository: incubator-streams
Updated Branches:
  refs/heads/master ae27541e0 -> c38eef6cf


Changes necessary to satisfy JIRA issues STREAMS-75,76,77 for Streams' URL Unwinder


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

Branch: refs/heads/master
Commit: 2d156348d34fccc56ffbaec12f4b1ca84895d77f
Parents: ae27541
Author: Matthew Hager <Ma...@gmail.com>
Authored: Thu May 8 04:41:25 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Thu May 8 04:41:25 2014 -0500

----------------------------------------------------------------------
 .../apache/streams/urls/DomainSensitivity.java  | 108 ++++++++++
 .../org/apache/streams/urls/LinkResolver.java   | 213 ++++++++++++-------
 .../streams/urls/LinkResolverProcessor.java     |  39 ++--
 .../streams/urls/TestDomainSensitivity.java     | 114 ++++++++++
 .../streams/urls/TestLinkUnwinderProcessor.java |   5 +-
 5 files changed, 371 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/2d156348/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
new file mode 100644
index 0000000..763e561
--- /dev/null
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -0,0 +1,108 @@
+package org.apache.streams.urls;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * This is a static utility helper class to obey domain sensitivity. It cannot be
+ * instantiated and can only be referenced through the static accessor functions
+ *
+ * @see
+ *
+ *
+ */
+public abstract class DomainSensitivity {
+
+    // The amount of time we want to space between domain calls
+    public static final long RECENT_DOMAINS_BACKOFF = 5000;
+    public static final long DEFAULT_STAGGER = RECENT_DOMAINS_BACKOFF / 10;
+
+    // Map to store the information of recent domains, with the last time they were accessed.
+    private static final ConcurrentMap<String, Date> RECENT_DOMAINS = new ConcurrentHashMap<String, Date>();
+
+    private static Timer timer;
+
+    public static void purgeAllDomainWaitTimes() {
+        RECENT_DOMAINS.clear();
+    }
+
+    public static long waitTimeForDomain(String domain) {
+        domain = domain.toLowerCase();
+
+        long toReturn = 0;
+        synchronized (DomainSensitivity.class) {
+            purgeAnyExpiredDomains();
+            // if the timer doesn't exist, then setup the timer (IE: first time called)
+            if(timer == null)
+                setupTimer();
+            long currentTime = new Date().getTime();
+
+            if(RECENT_DOMAINS.containsKey(domain)) {
+                // find the time it wants us to wait until
+                long nextTime = RECENT_DOMAINS.get(domain).getTime();
+                long random = (long)((Math.random() * (RECENT_DOMAINS_BACKOFF / 5))); // stagger
+
+                // back-off has been satisfied
+                if(currentTime >= nextTime)
+                    RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
+                else {
+                    // we are going to have to wait longer than the back-off
+                    // add the time we told them they needed to wait
+                    toReturn = (nextTime - currentTime) + RECENT_DOMAINS_BACKOFF;
+                    RECENT_DOMAINS.put(domain, new Date(currentTime + toReturn));
+                    toReturn += random + 1;
+                }
+            } else {
+                // no wait
+                RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
+            }
+        } // end synchronized block
+
+        return toReturn;
+    }
+
+    /**
+     * Quick function to setup the daemon to clear domains to keep our memory foot-print low
+     */
+    private static void setupTimer() {
+        timer = new Timer(true);
+        timer.schedule(new TimerTask() {
+            public void run() {
+                purgeAnyExpiredDomains();
+            }
+        }, RECENT_DOMAINS_BACKOFF * 2);
+    }
+
+    /**
+     * called by the timer to expire any domains
+     */
+    private static void purgeAnyExpiredDomains() {
+        // ensure this method is synchronized to get the proper information
+        synchronized (RECENT_DOMAINS) {
+            // figure the time that we would like for these domains to expire
+            long currentTime = new Date().getTime();
+            // see if there is any work that 'can' be done
+            if(RECENT_DOMAINS.size() != 0) {
+                // create a temporary list of the items that can be removed
+                Collection<String> ableToRemove = new HashSet<String>();
+
+
+                // iterate through all the domains (keys)
+                // if it qualifies, we can add it to the remove list
+                for(String k : RECENT_DOMAINS.keySet())
+                    if(currentTime >= RECENT_DOMAINS.get(k).getTime())
+                        ableToRemove.add(k);
+
+                if(ableToRemove.size() > 0)         // if there are domains to remove, then remove them
+                    for(String k : ableToRemove) // iterate through every domain that we can remove
+                        RECENT_DOMAINS.remove(k);   // remove the domain from our map.
+
+                for(String k : ableToRemove)
+                    System.out.println("Removing: " + k);
+
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/2d156348/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
index 5a4d224..f42342b 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
@@ -14,31 +14,43 @@ import java.net.URL;
 import java.net.URLDecoder;
 import java.util.*;
 
-/**
- * References:
- * Some helpful references to help
- * Purpose              URL
- * -------------        ----------------------------------------------------------------
- * [Status Codes]       http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html
- * [Test Cases]         http://greenbytes.de/tech/tc/httpredirects/
- * [t.co behavior]      https://dev.twitter.com/docs/tco-redirection-behavior
- */
-
-public class LinkResolver implements Serializable
-{
+public class LinkResolver implements Serializable {
+
+    /**
+     * References:
+     * Some helpful references to demonstrate the different types of browser re-directs that
+     * can happen. If you notice a redirect that was not followed to the proper place please
+     * submit a bug at :
+     * https://issues.apache.org/jira/browse/STREAMS
+     * <p/>
+     * Purpose              URL
+     * -------------        ----------------------------------------------------------------
+     * [Status Codes]       http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html
+     * [Test Cases]         http://greenbytes.de/tech/tc/httpredirects/
+     * [t.co behavior]      https://dev.twitter.com/docs/tco-redirection-behavior
+     */
+
+
     private final static Logger LOGGER = LoggerFactory.getLogger(LinkResolver.class);
 
-    private static final int MAX_ALLOWED_REDIRECTS = 30;
-    private static final int DEFAULT_HTTP_TIMEOUT = 5000; //originally 30000
+    private static final int MAX_ALLOWED_REDIRECTS = 30;                // We will only chase the link to it's final destination a max of 30 times.
+    private static final int DEFAULT_HTTP_TIMEOUT = 10000;              // We will only wait a max of 10,000 milliseconds (10 seconds) for any HTTP response
     private static final String LOCATION_IDENTIFIER = "location";
     private static final String SET_COOKIE_IDENTIFIER = "set-cookie";
 
-    private LinkDetails linkDetails = new LinkDetails();
+    // if Bots are not 'ok' this is the spoof settings that we'll use
+    private static final Map<String, String> SPOOF_HTTP_HEADERS = new HashMap<String, String>() {{
+        put("Connection", "Keep-Alive");
+        put("User-Agent", "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_0) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.48 Safari/537.36");
+        put("Accept-Language", "en-US,en;q=0.8,zh;q=0.6");
+    }};
 
+    // These are the known domains that are 'bot' friendly.
     private static final Collection<String> BOTS_ARE_OK = new ArrayList<String>() {{
-       add("t.co");
+        add("t.co");
     }};
 
+    // To help canonicalize the URL, these parts are 'known' to be 'ok' to remove
     private static final Collection<String> URL_TRACKING_TO_REMOVE = new ArrayList<String>() {{
         /******************************************************************
          * Google uses parameters in the URL string to track referrers
@@ -65,9 +77,28 @@ public class LinkResolver implements Serializable
         add("([\\?&])utm_campaign(=)[^&?]*");
     }};
 
-    public LinkDetails getLinkDetails()     { return linkDetails; }
 
+    // This element holds all the information about all the re-directs that have taken place
+    // and the steps and HTTP codes that occurred inside of each step.
+    private final LinkDetails linkDetails;
+    private Collection<String> domainsSensitiveTo = new HashSet<String>();
+
+    /**
+     * Get the link details
+     *
+     * @return Detailed log of every redirection that took place with the browser along with it it's ultimate status code.
+     */
+    public LinkDetails getLinkDetails() {
+        return linkDetails;
+    }
+
+    /**
+     * Raw string input of the URL. If the URL is invalid, the response code that is returned will indicate such.
+     *
+     * @param originalURL The URL you wish to unwind represented as a string.
+     */
     public LinkResolver(String originalURL) {
+        linkDetails = new LinkDetails();
         linkDetails.setOriginalURL(originalURL);
     }
 
@@ -78,8 +109,8 @@ public class LinkResolver implements Serializable
         linkDetails.setStartTime(DateTime.now());
         // we are going to try three times just incase we catch the service off-guard
         // this is mainly to help us with our tests.
-        for(int i = 0; (i < 3) && linkDetails.getFinalURL() == null ; i++) {
-            if(linkDetails.getLinkStatus() != LinkDetails.LinkStatus.SUCCESS)
+        for (int i = 0; (i < 3) && linkDetails.getFinalURL() == null; i++) {
+            if (linkDetails.getLinkStatus() != LinkDetails.LinkStatus.SUCCESS)
                 unwindLink(linkDetails.getOriginalURL());
         }
 
@@ -95,73 +126,97 @@ public class LinkResolver implements Serializable
         linkDetails.setTookInMills(DateTime.now().minus(linkDetails.getStartTime().getMillis()).getMillis());
     }
 
-    public void unwindLink(String url)
-    {
+    public void unwindLink(String url) {
         Preconditions.checkNotNull(linkDetails);
+        Preconditions.checkNotNull(url);
 
-        // Check to see if they wound up in a redirect loop
-        if((linkDetails.getRedirectCount() != null && linkDetails.getRedirectCount().longValue() > 0 && (linkDetails.getOriginalURL().equals(url) || linkDetails.getRedirects().contains(url))) || (linkDetails.getRedirectCount().longValue() > MAX_ALLOWED_REDIRECTS))
-        {
+        // Check to see if they wound up in a redirect loop,
+        // IE: 'A' redirects to 'B', then 'B' redirects to 'A'
+        if ((linkDetails.getRedirectCount() != null && linkDetails.getRedirectCount() > 0 &&
+                (linkDetails.getOriginalURL().equals(url) || linkDetails.getRedirects().contains(url)))
+                || (linkDetails.getRedirectCount() != null && linkDetails.getRedirectCount() > MAX_ALLOWED_REDIRECTS)) {
             linkDetails.setLinkStatus(LinkDetails.LinkStatus.LOOP);
             return;
         }
 
-        if(!linkDetails.getOriginalURL().equals(url))
+        if (!linkDetails.getOriginalURL().equals(url))
             linkDetails.getRedirects().add(url);
 
         HttpURLConnection connection = null;
 
-        try
-        {
+        // Store where the redirected link will go (if there is one)
+        String reDirectedLink = null;
+
+        try {
+            // Turn the string into a URL
             URL thisURL = new URL(url);
-            connection = (HttpURLConnection)new URL(url).openConnection();
+
+            // Be sensitive to overloading domains STREAMS-77
+            try {
+                String host = thisURL.getHost().toLowerCase();
+                if(!domainsSensitiveTo.contains(host)) {
+                    domainsSensitiveTo.add(host);
+                    long domainWait = DomainSensitivity.waitTimeForDomain(thisURL.getHost());
+                    if (domainWait > 0) {
+                        LOGGER.debug("Waiting for domain: {}", domainWait);
+                        Thread.sleep(domainWait);
+                    }
+                }
+            } catch(Exception e) {
+                // noOp
+            }
+
+            connection = (HttpURLConnection) new URL(url).openConnection();
 
             // now we are going to pretend that we are a browser...
             // This is the way my mac works.
-            if(!BOTS_ARE_OK.contains(thisURL.getHost()))
-            {
+            if (!BOTS_ARE_OK.contains(thisURL.getHost())) {
                 connection.addRequestProperty("Host", thisURL.getHost());
-                connection.addRequestProperty("Connection", "Keep-Alive");
-                connection.addRequestProperty("User-Agent", "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_0) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.48 Safari/537.36");
-                connection.addRequestProperty("Accept-Language", "en-US,en;q=0.8,zh;q=0.6");
+
+                // Bots are not 'ok', so we need to spoof the headers
+                for (String k : SPOOF_HTTP_HEADERS.keySet())
+                    connection.addRequestProperty(k, SPOOF_HTTP_HEADERS.get(k));
 
                 // the test to seattlemamadoc.com prompted this change.
                 // they auto detect bots by checking the referrer chain and the 'user-agent'
                 // this broke the t.co test. t.co URLs are EXPLICITLY ok with bots
                 // there is a list for URLS that behave this way at the top in BOTS_ARE_OK
                 // smashew 2013-13-2013
-
-                if(linkDetails.getRedirectCount() > 0 && BOTS_ARE_OK.contains(thisURL.getHost()))
+                if (linkDetails.getRedirectCount() > 0 && BOTS_ARE_OK.contains(thisURL.getHost()))
                     connection.addRequestProperty("Referrer", linkDetails.getOriginalURL());
             }
 
             connection.setReadTimeout(DEFAULT_HTTP_TIMEOUT);
             connection.setConnectTimeout(DEFAULT_HTTP_TIMEOUT);
 
+            // we want to follow this behavior on our own to ensure that we are getting to the
+            // proper place. This is especially true with links that are wounded by special
+            // link winders,
+            // IE:
             connection.setInstanceFollowRedirects(false);
 
-            if(linkDetails.getCookies() != null)
+            if (linkDetails.getCookies() != null)
                 for (String cookie : linkDetails.getCookies())
                     connection.addRequestProperty("Cookie", cookie.split(";", 1)[0]);
 
             connection.connect();
 
-            linkDetails.setFinalResponseCode((long)connection.getResponseCode());
+            linkDetails.setFinalResponseCode((long) connection.getResponseCode());
 
-            /**************
-             *
-             */
-            Map<String,List<String>> headers = createCaseInsensitiveMap(connection.getHeaderFields());
+            Map<String, List<String>> headers = createCaseInsensitiveMap(connection.getHeaderFields());
             /******************************************************************
              * If they want us to set cookies, well, then we will set cookies
              * Example URL:
              * http://nyti.ms/1bCpesx
              *****************************************************************/
-            if(headers.containsKey(SET_COOKIE_IDENTIFIER))
+            if (headers.containsKey(SET_COOKIE_IDENTIFIER))
                 linkDetails.getCookies().add(headers.get(SET_COOKIE_IDENTIFIER).get(0));
 
-            switch (linkDetails.getFinalResponseCode().intValue())
-            {
+            switch (linkDetails.getFinalResponseCode().intValue()) {
+                /**
+                 * W3C HTTP Response Codes:
+                 * http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html
+                 */
                 case 200: // HTTP OK
                     linkDetails.setFinalURL(connection.getURL().toString());
                     linkDetails.setDomain(new URL(linkDetails.getFinalURL()).getHost());
@@ -190,18 +245,15 @@ public class LinkResolver implements Serializable
                      * still render the page with it's content, but for us to assert
                      * a success, we are really hoping for a 304 message.
                      *******************************************************************/
-                    if(!linkDetails.getOriginalURL().toLowerCase().equals(connection.getURL().toString().toLowerCase()))
+                    if (!linkDetails.getOriginalURL().toLowerCase().equals(connection.getURL().toString().toLowerCase()))
                         linkDetails.setFinalURL(connection.getURL().toString());
-                    if(!headers.containsKey(LOCATION_IDENTIFIER))
-                    {
+                    if (!headers.containsKey(LOCATION_IDENTIFIER)) {
                         LOGGER.info("Headers: {}", headers);
                         linkDetails.setLinkStatus(LinkDetails.LinkStatus.REDIRECT_ERROR);
-                    }
-                    else
-                    {
+                    } else {
                         linkDetails.setRedirected(Boolean.TRUE);
-                        linkDetails.setRedirectCount(linkDetails.getRedirectCount().longValue()+1);
-                        unwindLink(connection.getHeaderField(LOCATION_IDENTIFIER));
+                        linkDetails.setRedirectCount(linkDetails.getRedirectCount() + 1);
+                        reDirectedLink = connection.getHeaderField(LOCATION_IDENTIFIER);
                     }
                     break;
                 case 305: // User must use the specified proxy (deprecated by W3C)
@@ -228,52 +280,51 @@ public class LinkResolver implements Serializable
                     linkDetails.setLinkStatus(LinkDetails.LinkStatus.NOT_FOUND);
                     break;
             }
-        }
-        catch (MalformedURLException e)
-        {
+        } catch (MalformedURLException e) {
             // the URL is trash, so, it can't load it.
             linkDetails.setLinkStatus(LinkDetails.LinkStatus.MALFORMED_URL);
-        }
-        catch (IOException ex)
-        {
+        } catch (IOException ex) {
             // there was an issue we are going to set to error.
             linkDetails.setLinkStatus(LinkDetails.LinkStatus.ERROR);
-        }
-        catch (Exception ex)
-        {
+        } catch (Exception ex) {
             // there was an unknown issue we are going to set to exception.
             linkDetails.setLinkStatus(LinkDetails.LinkStatus.EXCEPTION);
-        }
-        finally
-        {
+        } finally {
+            // if the connection is not null, then we need to disconnect to close any underlying resources
             if (connection != null)
                 connection.disconnect();
         }
+
+        // If there was a redirection, then we have to keep going
+        // Placing this code here should help to satisfy ensuring that the connection object
+        // is closed successfully.
+        if (reDirectedLink != null)
+            unwindLink(reDirectedLink);
+
     }
 
-    private Map<String,List<String>> createCaseInsensitiveMap(Map<String,List<String>> input) {
-        Map<String,List<String>> toReturn = new HashMap<String, List<String>>();
-        for(String k : input.keySet())
-            if(k != null && input.get(k) != null)
+    private Map<String, List<String>> createCaseInsensitiveMap(Map<String, List<String>> input) {
+        Map<String, List<String>> toReturn = new HashMap<String, List<String>>();
+        for (String k : input.keySet())
+            if (k != null && input.get(k) != null)
                 toReturn.put(k.toLowerCase(), input.get(k));
         return toReturn;
     }
 
-    private String cleanURL(String url)
-    {
+    private String cleanURL(String url) {
         // If they pass us a null URL then we are going to pass that right back to them.
-        if(url == null)
+        if (url == null)
             return null;
 
         // remember how big the URL was at the start
         int startLength = url.length();
 
         // Iterate through all the known URL parameters of tracking URLs
-        for(String pattern : URL_TRACKING_TO_REMOVE)
+        for (String pattern : URL_TRACKING_TO_REMOVE)
             url = url.replaceAll(pattern, "");
 
         // If the URL is smaller than when it came in. Then it had tracking information
-        if(url.length() < startLength)
+        if (url.length() < startLength)
             linkDetails.setTracked(Boolean.TRUE);
 
         // return our url.
@@ -284,11 +335,11 @@ public class LinkResolver implements Serializable
      * Removes the protocol, if it exists, from the front and
      * removes any random encoding characters
      * Extend this to do other url cleaning/pre-processing
+     *
      * @param url - The String URL to normalize
      * @return normalizedUrl - The String URL that has no junk or surprises
      */
-    public static String normalizeURL(String url)
-    {
+    public static String normalizeURL(String url) {
         // Decode URL to remove any %20 type stuff
         String normalizedUrl = url;
         try {
@@ -296,9 +347,7 @@ public class LinkResolver implements Serializable
             // but this functionality exists in most modern languages
             // and is universally called url decoding
             normalizedUrl = URLDecoder.decode(url, "UTF-8");
-        }
-        catch(UnsupportedEncodingException uee)
-        {
+        } catch (UnsupportedEncodingException uee) {
             System.err.println("Unable to Decode URL. Decoding skipped.");
             uee.printStackTrace();
         }
@@ -315,14 +364,14 @@ public class LinkResolver implements Serializable
     /**
      * Goal is to get the different parts of the URL path. This can be used
      * in a classifier to help us determine if we are working with
-     *
+     * <p/>
      * Reference:
      * http://stackoverflow.com/questions/10046178/pattern-matching-for-url-classification
+     *
      * @param url - Url to be tokenized
      * @return tokens - A String array of all the tokens
      */
-    public static List<String> tokenizeURL(String url)
-    {
+    public static List<String> tokenizeURL(String url) {
         url = normalizeURL(url);
         // I assume that we're going to use the whole URL to find tokens in
         // If you want to just look in the GET parameters, or you want to ignore the domain
@@ -333,7 +382,7 @@ public class LinkResolver implements Serializable
         // Split the URL by forward slashes. Most modern browsers will accept a URL
         // this malformed such as http://www.smashew.com/hello//how////are/you
         // hence the '+' in the regular expression.
-        for(String part: url.split("/+"))
+        for (String part : url.split("/+"))
             toReturn.add(part.toLowerCase());
 
         // return our object.

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/2d156348/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
index 929d7cd..c2cfe7d 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
@@ -11,22 +11,10 @@ import org.slf4j.LoggerFactory;
 
 import java.util.*;
 
-/**
- * References:
- * Some helpful references to help
- * Purpose              URL
- * -------------        ----------------------------------------------------------------
- * [Status Codes]       http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html
- * [Test Cases]         http://greenbytes.de/tech/tc/httpredirects/
- * [t.co behavior]      https://dev.twitter.com/docs/tco-redirection-behavior
- */
-
-public class LinkResolverProcessor implements StreamsProcessor
-{
-    private final static String STREAMS_ID = "LinkResolverProcessor";
-
-    private final static Logger LOGGER = LoggerFactory.getLogger(LinkResolverProcessor.class);
+public class LinkResolverProcessor implements StreamsProcessor {
 
+    private static final String STREAMS_ID = "LinkResolverProcessor";
+    private static final Logger LOGGER = LoggerFactory.getLogger(LinkResolverProcessor.class);
     private static ObjectMapper mapper = StreamsJacksonMapper.getInstance();
 
     @Override
@@ -39,7 +27,7 @@ public class LinkResolverProcessor implements StreamsProcessor
         Activity activity;
 
         // get list of shared urls
-        if( entry.getDocument() instanceof Activity) {
+        if (entry.getDocument() instanceof Activity) {
             activity = (Activity) entry.getDocument();
 
             activity.setLinks(unwind(activity.getLinks()));
@@ -49,14 +37,14 @@ public class LinkResolverProcessor implements StreamsProcessor
             result.add(entry);
 
             return result;
-        } else if( entry.getDocument() instanceof String ) {
+        } else if (entry.getDocument() instanceof String) {
 
             try {
                 activity = mapper.readValue((String) entry.getDocument(), Activity.class);
             } catch (Exception e) {
                 e.printStackTrace();
                 LOGGER.warn(e.getMessage());
-                return(Lists.newArrayList(entry));
+                return (Lists.newArrayList(entry));
             }
 
             activity.setLinks(unwind(activity.getLinks()));
@@ -66,32 +54,33 @@ public class LinkResolverProcessor implements StreamsProcessor
             } catch (Exception e) {
                 e.printStackTrace();
                 LOGGER.warn(e.getMessage());
-                return(Lists.newArrayList());
+                return (Lists.newArrayList());
             }
 
             result.add(entry);
 
             return result;
 
-        }
-        else {
+        } else {
             //return(Lists.newArrayList(entry));
-            return( Lists.newArrayList());
+            return (Lists.newArrayList());
         }
     }
 
     @Override
     public void prepare(Object o) {
+        // noOp
     }
 
     @Override
     public void cleanUp() {
-
+        // noOp
     }
 
-    private List<String> unwind(List<String> inputLinks) {
+
+    protected List<String> unwind(List<String> inputLinks) {
         List<String> outputLinks = Lists.newArrayList();
-        for( String link : inputLinks ) {
+        for (String link : inputLinks) {
             try {
                 LinkResolver unwinder = new LinkResolver(link);
                 unwinder.run();

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/2d156348/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
new file mode 100644
index 0000000..356404b
--- /dev/null
+++ b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
@@ -0,0 +1,114 @@
+package org.apache.streams.urls;
+
+import org.junit.Test;
+
+import java.util.Date;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+
+public class TestDomainSensitivity {
+
+    @Test
+    public void testSimple() {
+
+        DomainSensitivity.purgeAllDomainWaitTimes();
+        String domain1 = "smashew.com";
+
+        // safe to run...
+        assertEquals("smashew.com: No need to wait", 0, DomainSensitivity.waitTimeForDomain(domain1));
+        // get required sleep
+        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
+        // sleep
+        System.out.println("Sleeping: " + new Date().getTime() + "-" + smashewSleepTime1);
+        safeSleep(smashewSleepTime1);
+        System.out.println("Slept For: " + new Date().getTime() + "-" + smashewSleepTime1);
+        // safe to run again
+        assertEquals("smashew.com: No need to wait", 0, DomainSensitivity.waitTimeForDomain(domain1));
+    }
+
+    private static void safeSleep(long millis) {
+        try {
+            Thread.sleep(millis);
+        }
+        catch(Exception e) {
+            // noOp
+        }
+    }
+
+    @Test
+    public void testSingle() {
+
+        DomainSensitivity.purgeAllDomainWaitTimes();
+
+        String domain1 = "smashew.com";
+        String domain2 = "google.com";
+
+        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
+        long smashewSleepTime2 = DomainSensitivity.waitTimeForDomain(domain1);
+        long smashewSleepTime3 = DomainSensitivity.waitTimeForDomain(domain1);
+        long smashewSleepTime4 = DomainSensitivity.waitTimeForDomain(domain1);
+
+        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3  + "," + smashewSleepTime4);
+
+        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
+        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER));
+        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 2) - (DomainSensitivity.DEFAULT_STAGGER * 2));
+        assertTrue("smashew.com: Wait for at least min x 3", smashewSleepTime4 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 3) - (DomainSensitivity.DEFAULT_STAGGER * 3));
+
+        long timeBeforeSleep = new Date().getTime();
+        System.out.println("Sleeping for: " + smashewSleepTime4 + " ms");
+
+        safeSleep(smashewSleepTime4);
+        System.out.println("Actually slept for: " + (new Date().getTime() - timeBeforeSleep) + " ms");
+
+        long postSleepDomain1 = DomainSensitivity.waitTimeForDomain(domain1);
+        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
+        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
+
+    }
+
+    @Test
+    public void testMulti() {
+
+        DomainSensitivity.purgeAllDomainWaitTimes();
+        String domain1 = "smashew.com";
+        String domain2 = "google.com";
+
+        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
+        long smashewSleepTime2 = DomainSensitivity.waitTimeForDomain(domain1);
+        long smashewSleepTime3 = DomainSensitivity.waitTimeForDomain(domain1);
+
+        long googleSleepTime1 = DomainSensitivity.waitTimeForDomain(domain2);
+        long googleSleepTime2 = DomainSensitivity.waitTimeForDomain(domain2);
+
+        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3);
+        System.out.println("google.com: " + googleSleepTime1 + "," + googleSleepTime2);
+
+        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
+        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER));
+        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 2) - (DomainSensitivity.DEFAULT_STAGGER * 2));
+
+        assertEquals("google.com: No need to wait", 0, googleSleepTime1);
+        assertTrue("google.com: No need to wait", googleSleepTime2 >= DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER);
+
+        try {
+            System.out.println("WAITING FOR: " + smashewSleepTime3);
+            Thread.sleep(smashewSleepTime3);
+        }
+        catch(Exception e) {
+            // noOp
+        }
+
+        long postSleepDomain1 = DomainSensitivity.waitTimeForDomain(domain1);
+        long postSleepDomain2 = DomainSensitivity.waitTimeForDomain(domain2);
+
+        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
+        System.out.println("google.com:  Post Sleep domain2: " + postSleepDomain2);
+
+        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
+        assertEquals("google.com: No need to wait after sleep", 0, postSleepDomain2);
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/2d156348/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
index 836b946..83658fe 100644
--- a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
+++ b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
@@ -8,10 +8,11 @@ import org.apache.streams.core.StreamsDatum;
 import org.apache.streams.jackson.StreamsJacksonModule;
 import org.apache.streams.pojo.json.Activity;
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 import java.util.List;
 
+import static org.junit.Assert.*;
+
 /**
  * Created by rebanks on 2/27/14.
  */
@@ -68,6 +69,7 @@ public class TestLinkUnwinderProcessor {
     }
 
     public void testActivityUnwinderHelper(List<String> input, List<String> expected) throws Exception{
+        DomainSensitivity.purgeAllDomainWaitTimes();
         ObjectMapper mapper = new ObjectMapper();
         mapper.disable(com.fasterxml.jackson.databind.SerializationFeature.WRITE_DATES_AS_TIMESTAMPS);
         mapper.registerModule(new StreamsJacksonModule());
@@ -90,6 +92,7 @@ public class TestLinkUnwinderProcessor {
     }
 
     public void testStringActivityUnwinderHelper(List<String> input, List<String> expected) throws Exception{
+        DomainSensitivity.purgeAllDomainWaitTimes();
         ObjectMapper mapper = new ObjectMapper();
         mapper.disable(com.fasterxml.jackson.databind.SerializationFeature.WRITE_DATES_AS_TIMESTAMPS);
         mapper.registerModule(new StreamsJacksonModule());


[2/9] git commit: sorry, changed the delay time to 1 second.

Posted by mf...@apache.org.
sorry, changed the delay time to 1 second.


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

Branch: refs/heads/master
Commit: 734e28532219bf5eb70d17022f5df958d7e93533
Parents: 2d15634
Author: Matthew Hager <Ma...@gmail.com>
Authored: Thu May 8 12:25:12 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Thu May 8 12:25:12 2014 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/streams/urls/DomainSensitivity.java    | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/734e2853/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
index 763e561..90a13fa 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -8,14 +8,11 @@ import java.util.concurrent.ConcurrentMap;
  * This is a static utility helper class to obey domain sensitivity. It cannot be
  * instantiated and can only be referenced through the static accessor functions
  *
- * @see
- *
- *
  */
 public abstract class DomainSensitivity {
 
     // The amount of time we want to space between domain calls
-    public static final long RECENT_DOMAINS_BACKOFF = 5000;
+    public static final long RECENT_DOMAINS_BACKOFF = 1000;
     public static final long DEFAULT_STAGGER = RECENT_DOMAINS_BACKOFF / 10;
 
     // Map to store the information of recent domains, with the last time they were accessed.


[7/9] git commit: good call now synchronized on the same object for readability. execution is not affected.

Posted by mf...@apache.org.
good call now synchronized on the same object for readability. execution is not affected.


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

Branch: refs/heads/master
Commit: ebc1938d97780231e1257f629aef87c22acf3c3d
Parents: 05d513f
Author: Matthew Hager <Ma...@gmail.com>
Authored: Mon May 12 12:57:35 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Mon May 12 12:57:35 2014 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/streams/urls/DomainSensitivity.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/ebc1938d/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
index 7db4874..ac97ba4 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -80,7 +80,7 @@ public abstract class DomainSensitivity {
      */
     private static void purgeAnyExpiredDomains() {
         // ensure this method is synchronized to get the proper information
-        synchronized (RECENT_DOMAINS) {
+        synchronized (DomainSensitivity.class) {
             // figure the time that we would like for these domains to expire
             long currentTime = new Date().getTime();
             // see if there is any work that 'can' be done


[9/9] git commit: added some more tests, added more personal utilities, and renamed DomainSensitivity to LinkResolverHelperFunctions to be more explicit since it has many more functions in it now.

Posted by mf...@apache.org.
added some more tests, added more personal utilities, and renamed DomainSensitivity to LinkResolverHelperFunctions to be more explicit since it has many more functions in it now.


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

Branch: refs/heads/master
Commit: c38eef6cf639ea542bc501fa7e1b38f51d704c41
Parents: 985ece5
Author: Matthew Hager <Ma...@gmail.com>
Authored: Mon May 12 13:28:34 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Mon May 12 13:28:34 2014 -0500

----------------------------------------------------------------------
 .../apache/streams/urls/DomainSensitivity.java  | 122 --------------
 .../org/apache/streams/urls/LinkResolver.java   |   2 +-
 .../urls/LinkResolverHelperFunctions.java       | 160 +++++++++++++++++++
 .../streams/urls/LinkHelperFunctionsTest.java   | 146 +++++++++++++++++
 .../streams/urls/TestDomainSensitivity.java     | 114 -------------
 .../streams/urls/TestLinkUnwinderProcessor.java |   4 +-
 6 files changed, 309 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
deleted file mode 100644
index 268f4ee..0000000
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ /dev/null
@@ -1,122 +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.urls;
-
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * This is a static utility helper class to obey domain sensitivity. It cannot be
- * instantiated and can only be referenced through the static accessor functions
- *
- */
-public abstract class DomainSensitivity {
-
-    // The amount of time we want to space between domain calls
-    public static final long RECENT_DOMAINS_BACKOFF = 1000;
-    public static final long DEFAULT_STAGGER = RECENT_DOMAINS_BACKOFF / 10;
-
-    // Map to store the information of recent domains, with the last time they were accessed.
-    private static final ConcurrentMap<String, Date> RECENT_DOMAINS = new ConcurrentHashMap<String, Date>();
-
-    private static Timer timer;
-
-    private DomainSensitivity() {
-        // force it not to be instantiated.
-    }
-
-    public static void purgeAllDomainWaitTimes() {
-        RECENT_DOMAINS.clear();
-    }
-
-    public static long waitTimeForDomain(String domain) {
-        domain = domain.toLowerCase();
-
-        long toReturn = 0;
-        synchronized (DomainSensitivity.class) {
-            purgeAnyExpiredDomains();
-            // if the timer doesn't exist, then setup the timer (IE: first time called)
-            if(timer == null)
-                setupTimer();
-            long currentTime = new Date().getTime();
-
-            if(RECENT_DOMAINS.containsKey(domain)) {
-                // find the time it wants us to wait until
-                long nextTime = RECENT_DOMAINS.get(domain).getTime();
-                long random = (long)((Math.random() * (RECENT_DOMAINS_BACKOFF / 5))); // stagger
-
-                // back-off has been satisfied
-                if(currentTime >= nextTime)
-                    RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
-                else {
-                    // we are going to have to wait longer than the back-off
-                    // add the time we told them they needed to wait
-                    toReturn = (nextTime - currentTime) + RECENT_DOMAINS_BACKOFF;
-                    RECENT_DOMAINS.put(domain, new Date(currentTime + toReturn));
-                    toReturn += random + 1;
-                }
-            } else {
-                // no wait
-                RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
-            }
-        } // end synchronized block
-
-        return toReturn;
-    }
-
-    /**
-     * Quick function to setup the daemon to clear domains to keep our memory foot-print low
-     */
-    private static void setupTimer() {
-        timer = new Timer(true);
-        timer.schedule(new TimerTask() {
-            public void run() {
-                purgeAnyExpiredDomains();
-            }
-        }, RECENT_DOMAINS_BACKOFF * 2);
-    }
-
-    /**
-     * called by the timer to expire any domains
-     */
-    private static void purgeAnyExpiredDomains() {
-        // ensure this method is synchronized to get the proper information
-        synchronized (DomainSensitivity.class) {
-            // figure the time that we would like for these domains to expire
-            long currentTime = new Date().getTime();
-            // see if there is any work that 'can' be done
-            if(RECENT_DOMAINS.size() != 0) {
-                // create a temporary list of the items that can be removed
-                Collection<String> ableToRemove = new HashSet<String>();
-
-
-                // iterate through all the domains (keys)
-                // if it qualifies, we can add it to the remove list
-                for(String k : RECENT_DOMAINS.keySet())
-                    if(currentTime >= RECENT_DOMAINS.get(k).getTime())
-                        ableToRemove.add(k);
-
-                if(ableToRemove.size() > 0)         // if there are domains to remove, then remove them
-                    for(String k : ableToRemove) // iterate through every domain that we can remove
-                        RECENT_DOMAINS.remove(k);   // remove the domain from our map.
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
index fa7bf56..154c2c6 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
@@ -177,7 +177,7 @@ public class LinkResolver implements Serializable {
                 String host = thisURL.getHost().toLowerCase();
                 if(!domainsSensitiveTo.contains(host)) {
                     domainsSensitiveTo.add(host);
-                    long domainWait = DomainSensitivity.waitTimeForDomain(thisURL.getHost());
+                    long domainWait = LinkResolverHelperFunctions.waitTimeForDomain(thisURL.getHost());
                     if (domainWait > 0) {
                         LOGGER.debug("Waiting for domain: {}", domainWait);
                         Thread.sleep(domainWait);

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverHelperFunctions.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverHelperFunctions.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverHelperFunctions.java
new file mode 100644
index 0000000..a551f84
--- /dev/null
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverHelperFunctions.java
@@ -0,0 +1,160 @@
+/*
+ * 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.urls;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
+
+/**
+ * This is a static utility helper class to verify strings are URLs,
+ * obey domain sensitivity, and find URLs within a string.
+ * It cannot be, instantiated and can only be referenced through
+ * the static accessor functions
+ *
+ */
+public final class LinkResolverHelperFunctions {
+
+    private static final String REGEX_URL =
+            "(?:(?:https?|ftp)://)" +                                               // protocol identifier
+                    "(?:\\S+(?::\\S*)?@)?" +                                                // user:pass authentication
+                    "(?:" +
+                    "(?!(?:10|127)(?:\\.\\d{1,3}){3})" +                                    // IP address exclusion
+                    "(?!(?:169\\.254|192\\.168)(?:\\.\\d{1,3}){2})" +                       // private & local networks
+                    "(?!172\\.(?:1[6-9]|2\\d|3[0-1])(?:\\.\\d{1,3}){2})" +
+                    "(?:[1-9]\\d?|1\\d\\d|2[01]\\d|22[0-3])" +                              // IP address dotted notation octets
+                    "(?:\\.(?:1?\\d{1,2}|2[0-4]\\d|25[0-5])){2}" +                          // excludes loop-back network 0.0.0.0, excludes network & broadcast addresses
+                    "(?:\\.(?:[1-9]\\d?|1\\d\\d|2[0-4]\\d|25[0-4]))" +                      // excludes reserved space >= 224.0.0.0, (first & last IP address of each class)
+                    "|" +
+                    "(?:(?:[a-z\\u00a1-\\uffff0-9]+-?)*[a-z\\u00a1-\\uffff0-9]+)" +         // host name
+                    "(?:\\.(?:[a-z\\u00a1-\\uffff0-9]+-?)*[a-z\\u00a1-\\uffff0-9]+)*" +     // domain name
+                    "(?:\\.(?:[a-z\\u00a1-\\uffff]{2,}))" +                                 // TLD identifier
+                    ")" +
+                    "(?::\\d{2,5})?" +                                                      // port number
+                    "(?:/[^\\s]*)?";                                                        // resource path
+
+    private static final String REGEX_URL_EXPLICIT = "^" + REGEX_URL + "$";
+
+    // The amount of time we want to space between domain calls
+    public static final long RECENT_DOMAINS_BACKOFF = 1000;
+    public static final long DEFAULT_STAGGER = RECENT_DOMAINS_BACKOFF / 10;
+
+    // Map to store the information of recent domains, with the last time they were accessed.
+    private static final ConcurrentMap<String, Date> RECENT_DOMAINS = new ConcurrentHashMap<String, Date>();
+
+    private static Timer timer;
+
+    /**
+     * Check to see if this string is a URL or not
+     * @param possibleURL
+     * The possible URL that we would like to test
+     * @return
+     * Whether or not it is a URL
+     */
+    public static boolean isURL(String possibleURL) {
+        return possibleURL.matches(REGEX_URL_EXPLICIT);
+    }
+
+    public static boolean containsURLs(String possiblyHasURLs) {
+        return possiblyHasURLs != null && Pattern.compile(REGEX_URL).matcher(possiblyHasURLs).find();
+    }
+
+    private LinkResolverHelperFunctions() {
+        // force it not to be instantiated.
+    }
+
+    public static void purgeAllDomainWaitTimes() {
+        RECENT_DOMAINS.clear();
+    }
+
+    public static long waitTimeForDomain(String domain) {
+        domain = domain.toLowerCase();
+
+        long toReturn = 0;
+        synchronized (LinkResolverHelperFunctions.class) {
+            purgeAnyExpiredDomains();
+            // if the timer doesn't exist, then setup the timer (IE: first time called)
+            if(timer == null)
+                setupTimer();
+            long currentTime = new Date().getTime();
+
+            if(RECENT_DOMAINS.containsKey(domain)) {
+                // find the time it wants us to wait until
+                long nextTime = RECENT_DOMAINS.get(domain).getTime();
+                long random = (long)((Math.random() * (RECENT_DOMAINS_BACKOFF / 5))); // stagger
+
+                // back-off has been satisfied
+                if(currentTime >= nextTime)
+                    RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
+                else {
+                    // we are going to have to wait longer than the back-off
+                    // add the time we told them they needed to wait
+                    toReturn = (nextTime - currentTime) + RECENT_DOMAINS_BACKOFF;
+                    RECENT_DOMAINS.put(domain, new Date(currentTime + toReturn));
+                    toReturn += random + 1;
+                }
+            } else {
+                // no wait
+                RECENT_DOMAINS.put(domain, new Date(currentTime + RECENT_DOMAINS_BACKOFF));
+            }
+        } // end synchronized block
+
+        return toReturn;
+    }
+
+    /**
+     * Quick function to setup the daemon to clear domains to keep our memory foot-print low
+     */
+    private static void setupTimer() {
+        timer = new Timer(true);
+        timer.schedule(new TimerTask() {
+            public void run() {
+                purgeAnyExpiredDomains();
+            }
+        }, RECENT_DOMAINS_BACKOFF * 2);
+    }
+
+    /**
+     * called by the timer to expire any domains
+     */
+    private static void purgeAnyExpiredDomains() {
+        // ensure this method is synchronized to get the proper information
+        synchronized (LinkResolverHelperFunctions.class) {
+            // figure the time that we would like for these domains to expire
+            long currentTime = new Date().getTime();
+            // see if there is any work that 'can' be done
+            if(RECENT_DOMAINS.size() != 0) {
+                // create a temporary list of the items that can be removed
+                Collection<String> ableToRemove = new HashSet<String>();
+
+
+                // iterate through all the domains (keys)
+                // if it qualifies, we can add it to the remove list
+                for(String k : RECENT_DOMAINS.keySet())
+                    if(currentTime >= RECENT_DOMAINS.get(k).getTime())
+                        ableToRemove.add(k);
+
+                if(ableToRemove.size() > 0)         // if there are domains to remove, then remove them
+                    for(String k : ableToRemove) // iterate through every domain that we can remove
+                        RECENT_DOMAINS.remove(k);   // remove the domain from our map.
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/LinkHelperFunctionsTest.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/LinkHelperFunctionsTest.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/LinkHelperFunctionsTest.java
new file mode 100644
index 0000000..407ff51
--- /dev/null
+++ b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/LinkHelperFunctionsTest.java
@@ -0,0 +1,146 @@
+package org.apache.streams.urls;
+
+import org.junit.Test;
+
+import java.util.Date;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertTrue;
+
+public class LinkHelperFunctionsTest {
+
+    @Test
+    public void testIsURL() {
+        assertTrue(LinkResolverHelperFunctions.isURL("http://goo.gl/wSrHDA"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://ow.ly/u4Kte"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://x.co/3yapt"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://bit.ly/1cX5Rh4"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://t.co/oP8JYB0MYW"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://goo.gl/wSrHDA"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://t.co/fBoCby3l1t"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://paper.li/GuyKawasaki"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://www.google.com"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://goo.gl/wSrHDA"));
+        assertTrue(LinkResolverHelperFunctions.isURL("http://www.cnn.com"));
+    }
+
+    @Test
+    public void testContainsURL() {
+        assertTrue(LinkResolverHelperFunctions.containsURLs("here is the URL: http://goo.gl/wSrHDA"));
+        assertTrue(LinkResolverHelperFunctions.containsURLs("a lovely day for URLing it up http://ow.ly/u4Kte"));
+        assertTrue(LinkResolverHelperFunctions.containsURLs("http://x.co/3yapt is really cool"));
+        assertTrue(LinkResolverHelperFunctions.containsURLs("http://bit.ly/1cX5Rh4 me likes"));
+        assertTrue(LinkResolverHelperFunctions.containsURLs("http://t.co/oP8JYB0MYW wtf mate?"));
+        assertTrue(LinkResolverHelperFunctions.containsURLs("Every morning is a good morning in URL world: http://goo.gl/wSrHDA"));
+
+        assertFalse(LinkResolverHelperFunctions.containsURLs("Every day I awake, only to find, I have no URLS"));
+        assertFalse(LinkResolverHelperFunctions.containsURLs("Http:// to be or not to be"));
+        assertFalse(LinkResolverHelperFunctions.containsURLs("Can I get an http://X up on the board pat?"));
+        assertFalse(LinkResolverHelperFunctions.containsURLs("You must remember Joey, no matter how much you ftp://stink you must never, EVER, take a shower in my dressing room!"));
+    }
+
+
+    @Test
+    public void testSimple() {
+
+        LinkResolverHelperFunctions.purgeAllDomainWaitTimes();
+        String domain1 = "smashew.com";
+
+        // safe to run...
+        assertEquals("smashew.com: No need to wait", 0, LinkResolverHelperFunctions.waitTimeForDomain(domain1));
+        // get required sleep
+        long smashewSleepTime1 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        // sleep
+        System.out.println("Sleeping: " + new Date().getTime() + "-" + smashewSleepTime1);
+        safeSleep(smashewSleepTime1);
+        System.out.println("Slept For: " + new Date().getTime() + "-" + smashewSleepTime1);
+        // safe to run again
+        assertEquals("smashew.com: No need to wait", 0, LinkResolverHelperFunctions.waitTimeForDomain(domain1));
+    }
+
+    private static void safeSleep(long millis) {
+        try {
+            Thread.sleep(millis);
+        }
+        catch(Exception e) {
+            // noOp
+        }
+    }
+
+    @Test
+    public void testSingle() {
+
+        LinkResolverHelperFunctions.purgeAllDomainWaitTimes();
+
+        String domain1 = "smashew.com";
+        String domain2 = "google.com";
+
+        long smashewSleepTime1 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long smashewSleepTime2 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long smashewSleepTime3 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long smashewSleepTime4 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+
+        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3  + "," + smashewSleepTime4);
+
+        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
+        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF - LinkResolverHelperFunctions.DEFAULT_STAGGER));
+        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF * 2) - (LinkResolverHelperFunctions.DEFAULT_STAGGER * 2));
+        assertTrue("smashew.com: Wait for at least min x 3", smashewSleepTime4 >= (LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF * 3) - (LinkResolverHelperFunctions.DEFAULT_STAGGER * 3));
+
+        long timeBeforeSleep = new Date().getTime();
+        System.out.println("Sleeping for: " + smashewSleepTime4 + " ms");
+
+        safeSleep(smashewSleepTime4);
+        System.out.println("Actually slept for: " + (new Date().getTime() - timeBeforeSleep) + " ms");
+
+        long postSleepDomain1 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
+        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
+
+    }
+
+    @Test
+    public void testMulti() {
+
+        LinkResolverHelperFunctions.purgeAllDomainWaitTimes();
+        String domain1 = "smashew.com";
+        String domain2 = "google.com";
+
+        long smashewSleepTime1 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long smashewSleepTime2 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long smashewSleepTime3 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+
+        long googleSleepTime1 = LinkResolverHelperFunctions.waitTimeForDomain(domain2);
+        long googleSleepTime2 = LinkResolverHelperFunctions.waitTimeForDomain(domain2);
+
+        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3);
+        System.out.println("google.com: " + googleSleepTime1 + "," + googleSleepTime2);
+
+        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
+        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF - LinkResolverHelperFunctions.DEFAULT_STAGGER));
+        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF * 2) - (LinkResolverHelperFunctions.DEFAULT_STAGGER * 2));
+
+        assertEquals("google.com: No need to wait", 0, googleSleepTime1);
+        assertTrue("google.com: No need to wait", googleSleepTime2 >= LinkResolverHelperFunctions.RECENT_DOMAINS_BACKOFF - LinkResolverHelperFunctions.DEFAULT_STAGGER);
+
+        try {
+            System.out.println("WAITING FOR: " + smashewSleepTime3);
+            Thread.sleep(smashewSleepTime3);
+        }
+        catch(Exception e) {
+            // noOp
+        }
+
+        long postSleepDomain1 = LinkResolverHelperFunctions.waitTimeForDomain(domain1);
+        long postSleepDomain2 = LinkResolverHelperFunctions.waitTimeForDomain(domain2);
+
+        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
+        System.out.println("google.com:  Post Sleep domain2: " + postSleepDomain2);
+
+        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
+        assertEquals("google.com: No need to wait after sleep", 0, postSleepDomain2);
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
deleted file mode 100644
index 356404b..0000000
--- a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestDomainSensitivity.java
+++ /dev/null
@@ -1,114 +0,0 @@
-package org.apache.streams.urls;
-
-import org.junit.Test;
-
-import java.util.Date;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertTrue;
-
-public class TestDomainSensitivity {
-
-    @Test
-    public void testSimple() {
-
-        DomainSensitivity.purgeAllDomainWaitTimes();
-        String domain1 = "smashew.com";
-
-        // safe to run...
-        assertEquals("smashew.com: No need to wait", 0, DomainSensitivity.waitTimeForDomain(domain1));
-        // get required sleep
-        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
-        // sleep
-        System.out.println("Sleeping: " + new Date().getTime() + "-" + smashewSleepTime1);
-        safeSleep(smashewSleepTime1);
-        System.out.println("Slept For: " + new Date().getTime() + "-" + smashewSleepTime1);
-        // safe to run again
-        assertEquals("smashew.com: No need to wait", 0, DomainSensitivity.waitTimeForDomain(domain1));
-    }
-
-    private static void safeSleep(long millis) {
-        try {
-            Thread.sleep(millis);
-        }
-        catch(Exception e) {
-            // noOp
-        }
-    }
-
-    @Test
-    public void testSingle() {
-
-        DomainSensitivity.purgeAllDomainWaitTimes();
-
-        String domain1 = "smashew.com";
-        String domain2 = "google.com";
-
-        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
-        long smashewSleepTime2 = DomainSensitivity.waitTimeForDomain(domain1);
-        long smashewSleepTime3 = DomainSensitivity.waitTimeForDomain(domain1);
-        long smashewSleepTime4 = DomainSensitivity.waitTimeForDomain(domain1);
-
-        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3  + "," + smashewSleepTime4);
-
-        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
-        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER));
-        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 2) - (DomainSensitivity.DEFAULT_STAGGER * 2));
-        assertTrue("smashew.com: Wait for at least min x 3", smashewSleepTime4 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 3) - (DomainSensitivity.DEFAULT_STAGGER * 3));
-
-        long timeBeforeSleep = new Date().getTime();
-        System.out.println("Sleeping for: " + smashewSleepTime4 + " ms");
-
-        safeSleep(smashewSleepTime4);
-        System.out.println("Actually slept for: " + (new Date().getTime() - timeBeforeSleep) + " ms");
-
-        long postSleepDomain1 = DomainSensitivity.waitTimeForDomain(domain1);
-        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
-        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
-
-    }
-
-    @Test
-    public void testMulti() {
-
-        DomainSensitivity.purgeAllDomainWaitTimes();
-        String domain1 = "smashew.com";
-        String domain2 = "google.com";
-
-        long smashewSleepTime1 = DomainSensitivity.waitTimeForDomain(domain1);
-        long smashewSleepTime2 = DomainSensitivity.waitTimeForDomain(domain1);
-        long smashewSleepTime3 = DomainSensitivity.waitTimeForDomain(domain1);
-
-        long googleSleepTime1 = DomainSensitivity.waitTimeForDomain(domain2);
-        long googleSleepTime2 = DomainSensitivity.waitTimeForDomain(domain2);
-
-        System.out.println("smashew.com: " + smashewSleepTime1 + "," + smashewSleepTime2 + "," + smashewSleepTime3);
-        System.out.println("google.com: " + googleSleepTime1 + "," + googleSleepTime2);
-
-        assertEquals("smashew.com: No need to wait", 0, smashewSleepTime1);
-        assertTrue("smashew.com: Wait for at least min x 1", smashewSleepTime2 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER));
-        assertTrue("smashew.com: Wait for at least min x 2", smashewSleepTime3 >= (DomainSensitivity.RECENT_DOMAINS_BACKOFF * 2) - (DomainSensitivity.DEFAULT_STAGGER * 2));
-
-        assertEquals("google.com: No need to wait", 0, googleSleepTime1);
-        assertTrue("google.com: No need to wait", googleSleepTime2 >= DomainSensitivity.RECENT_DOMAINS_BACKOFF - DomainSensitivity.DEFAULT_STAGGER);
-
-        try {
-            System.out.println("WAITING FOR: " + smashewSleepTime3);
-            Thread.sleep(smashewSleepTime3);
-        }
-        catch(Exception e) {
-            // noOp
-        }
-
-        long postSleepDomain1 = DomainSensitivity.waitTimeForDomain(domain1);
-        long postSleepDomain2 = DomainSensitivity.waitTimeForDomain(domain2);
-
-        System.out.println("smashew.com: Post Sleep domain1: " + postSleepDomain1);
-        System.out.println("google.com:  Post Sleep domain2: " + postSleepDomain2);
-
-        assertEquals("Smashew.com: No need to wait after sleep", 0, postSleepDomain1);
-        assertEquals("google.com: No need to wait after sleep", 0, postSleepDomain2);
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/c38eef6c/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
index 6d75687..93bbfb8 100644
--- a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
+++ b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
@@ -69,7 +69,7 @@ public class TestLinkUnwinderProcessor {
     }
 
     public void testActivityUnwinderHelper(List<String> input, List<String> expected) throws Exception{
-        DomainSensitivity.purgeAllDomainWaitTimes();
+        LinkResolverHelperFunctions.purgeAllDomainWaitTimes();
         ObjectMapper mapper = new ObjectMapper();
         mapper.disable(com.fasterxml.jackson.databind.SerializationFeature.WRITE_DATES_AS_TIMESTAMPS);
         mapper.registerModule(new StreamsJacksonModule());
@@ -92,7 +92,7 @@ public class TestLinkUnwinderProcessor {
     }
 
     public void testStringActivityUnwinderHelper(List<String> input, List<String> expected) throws Exception{
-        DomainSensitivity.purgeAllDomainWaitTimes();
+        LinkResolverHelperFunctions.purgeAllDomainWaitTimes();
         ObjectMapper mapper = new ObjectMapper();
         mapper.disable(com.fasterxml.jackson.databind.SerializationFeature.WRITE_DATES_AS_TIMESTAMPS);
         mapper.registerModule(new StreamsJacksonModule());


[3/9] git commit: Changed this test for speed, has no effect on the outcome.

Posted by mf...@apache.org.
Changed this test for speed, has no effect on the outcome.


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

Branch: refs/heads/master
Commit: 7115af06f49d511219d226a5496c62cc32648892
Parents: 734e285
Author: Matthew Hager <Ma...@gmail.com>
Authored: Thu May 8 13:53:05 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Thu May 8 13:53:05 2014 -0500

----------------------------------------------------------------------
 .../java/org/apache/streams/urls/TestLinkUnwinderProcessor.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/7115af06/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
index 83658fe..6d75687 100644
--- a/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
+++ b/streams-contrib/streams-processor-urls/src/test/java/org/apache/streams/urls/TestLinkUnwinderProcessor.java
@@ -64,8 +64,8 @@ public class TestLinkUnwinderProcessor {
 
     @Test
     public void testActivityLinkUnwinderProcessorUnwindable() throws Exception{
-        testActivityUnwinderHelper(Lists.newArrayList("http://bit.ly/1cX5Rh4", "http://nope@#$%"), Lists.newArrayList("http://www.wcgworld.com/"));
-        testStringActivityUnwinderHelper(Lists.newArrayList("http://bit.ly/1cX5Rh4", "http://nope@#$%"), Lists.newArrayList("http://www.wcgworld.com/"));
+        testActivityUnwinderHelper(Lists.newArrayList("http://bit.ly/1cX5Rh4", "http://nope@#$%e"), Lists.newArrayList("http://www.wcgworld.com/"));
+        testStringActivityUnwinderHelper(Lists.newArrayList("http://bit.ly/1cX5Rh4", "http://nope@#$%s"), Lists.newArrayList("http://www.wcgworld.com/"));
     }
 
     public void testActivityUnwinderHelper(List<String> input, List<String> expected) throws Exception{


[4/9] git commit: Removing debug statements.

Posted by mf...@apache.org.
Removing debug statements.


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

Branch: refs/heads/master
Commit: 9681392c98f0f21f5021fad43e15c169abe9fff8
Parents: 7115af0
Author: Matthew Hager <Ma...@gmail.com>
Authored: Thu May 8 20:33:21 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Thu May 8 20:33:21 2014 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/streams/urls/DomainSensitivity.java | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/9681392c/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
index 90a13fa..affc7f4 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -94,10 +94,6 @@ public abstract class DomainSensitivity {
                 if(ableToRemove.size() > 0)         // if there are domains to remove, then remove them
                     for(String k : ableToRemove) // iterate through every domain that we can remove
                         RECENT_DOMAINS.remove(k);   // remove the domain from our map.
-
-                for(String k : ableToRemove)
-                    System.out.println("Removing: " + k);
-
             }
         }
     }


[6/9] git commit: force this file not to be instanciated

Posted by mf...@apache.org.
force this file not to be instanciated


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

Branch: refs/heads/master
Commit: 05d513f244a801b7342410b89aa20fbd7e7822c6
Parents: d2a8aa0
Author: Matthew Hager <Ma...@gmail.com>
Authored: Mon May 12 12:54:33 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Mon May 12 12:54:33 2014 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/streams/urls/DomainSensitivity.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/05d513f2/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
index affc7f4..7db4874 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -20,6 +20,10 @@ public abstract class DomainSensitivity {
 
     private static Timer timer;
 
+    private DomainSensitivity() {
+        // force it not to be instantiated.
+    }
+
     public static void purgeAllDomainWaitTimes() {
         RECENT_DOMAINS.clear();
     }


[5/9] git commit: While running this, I found a null bug that prevented the flag for "wasRedirected" to never be set. This corrects that issue.

Posted by mf...@apache.org.
While running this, I found a null bug that prevented the flag for "wasRedirected" to never be set. This corrects that issue.


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

Branch: refs/heads/master
Commit: d2a8aa046dc01e4105aa436cbd1cf38ef960e75b
Parents: 9681392
Author: Matthew Hager <Ma...@gmail.com>
Authored: Thu May 8 22:51:46 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Thu May 8 22:51:46 2014 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/streams/urls/LinkResolver.java | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/d2a8aa04/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
index f42342b..358eebd 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
@@ -107,12 +107,16 @@ public class LinkResolver implements Serializable {
         Preconditions.checkNotNull(linkDetails.getOriginalURL());
 
         linkDetails.setStartTime(DateTime.now());
-        // we are going to try three times just incase we catch the service off-guard
-        // this is mainly to help us with our tests.
-        for (int i = 0; (i < 3) && linkDetails.getFinalURL() == null; i++) {
+
+        // we are going to try three times just in case we catch a slow server or one that needs
+        // to be warmed up. This tends to happen many times with smaller private servers
+        for (int i = 0; (i < 3) && linkDetails.getFinalURL() == null; i++)
             if (linkDetails.getLinkStatus() != LinkDetails.LinkStatus.SUCCESS)
                 unwindLink(linkDetails.getOriginalURL());
-        }
+
+        // because this is a POJO we need to make sure that we set this to false if it was never re-directed
+        if(this.linkDetails.getRedirectCount() == 0 || this.linkDetails.getRedirected() == null)
+            this.linkDetails.setRedirected(false);
 
         linkDetails.setFinalURL(cleanURL(linkDetails.getFinalURL()));
         linkDetails.setNormalizedURL(normalizeURL(linkDetails.getFinalURL()));


[8/9] git commit: Added license headers.

Posted by mf...@apache.org.
Added license headers.


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

Branch: refs/heads/master
Commit: 985ece5f5d79ea111ce185669428b7245f0eb4bc
Parents: ebc1938
Author: Matthew Hager <Ma...@gmail.com>
Authored: Mon May 12 13:01:25 2014 -0500
Committer: Matthew Hager <Ma...@gmail.com>
Committed: Mon May 12 13:01:25 2014 -0500

----------------------------------------------------------------------
 .../org/apache/streams/urls/DomainSensitivity.java | 17 +++++++++++++++++
 .../java/org/apache/streams/urls/LinkResolver.java | 17 +++++++++++++++++
 .../apache/streams/urls/LinkResolverProcessor.java | 17 +++++++++++++++++
 3 files changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/985ece5f/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
index ac97ba4..268f4ee 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/DomainSensitivity.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *
+ *   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.urls;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/985ece5f/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
index 358eebd..fa7bf56 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolver.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *
+ *   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.urls;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/985ece5f/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
----------------------------------------------------------------------
diff --git a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
index c2cfe7d..ec94c88 100644
--- a/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
+++ b/streams-contrib/streams-processor-urls/src/main/java/org/apache/streams/urls/LinkResolverProcessor.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *
+ *   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.urls;
 
 import com.fasterxml.jackson.databind.ObjectMapper;