You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by hu...@apache.org on 2017/10/14 02:12:43 UTC

incubator-gobblin git commit: [GOBBLIN-288] Add finer-grain dynamic partition generation for Salesf…

Repository: incubator-gobblin
Updated Branches:
  refs/heads/master 07c86f2a7 -> 626d312a2


[GOBBLIN-288] Add finer-grain dynamic partition generation for Salesf…

Closes #2140 from
htran1/salesforce_dynamic_probing


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

Branch: refs/heads/master
Commit: 626d312a2c8ee70f92520efc190e552adb808196
Parents: 07c86f2
Author: Hung Tran <hu...@linkedin.com>
Authored: Fri Oct 13 19:12:36 2017 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Oct 13 19:12:36 2017 -0700

----------------------------------------------------------------------
 .../gobblin/source/extractor/utils/Utils.java   |  11 +
 .../gobblin/salesforce/SalesforceSource.java    | 290 ++++++++++++++++---
 .../salesforce/SalesforceSourceTest.java        |   7 +-
 3 files changed, 266 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/626d312a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/utils/Utils.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/utils/Utils.java b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/utils/Utils.java
index 7496268..86d8f58 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/utils/Utils.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/utils/Utils.java
@@ -112,6 +112,17 @@ public class Utils {
     return outDate;
   }
 
+  public static Date toDate(String input, String inputfmt) {
+    final SimpleDateFormat inputFormat = new SimpleDateFormat(inputfmt);
+    Date outDate = null;
+    try {
+      outDate = inputFormat.parse(input);
+    } catch (ParseException e) {
+      LOG.error("Parse to date failed", e);
+    }
+    return outDate;
+  }
+
   public static String epochToDate(long epoch, String format) {
     SimpleDateFormat sdf = new SimpleDateFormat(format);
     Date date = new Date(epoch);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/626d312a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceSource.java
----------------------------------------------------------------------
diff --git a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceSource.java b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceSource.java
index 6089415..b2de3da 100644
--- a/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceSource.java
+++ b/gobblin-salesforce/src/main/java/org/apache/gobblin/salesforce/SalesforceSource.java
@@ -65,6 +65,7 @@ import org.apache.gobblin.source.workunit.WorkUnit;
 
 import lombok.AllArgsConstructor;
 import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 
 
@@ -78,9 +79,25 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
   public static final boolean DEFAULT_USE_ALL_OBJECTS = false;
 
   private static final String ENABLE_DYNAMIC_PARTITIONING = "salesforce.enableDynamicPartitioning";
-  private static final String DAY_PARTITION_QUERY_TEMPLATE = "SELECT count(${column}) cnt, DAY_ONLY(${column}) time FROM ${table} "
-      + "WHERE ${column} ${greater} ${start} AND ${column} ${less} ${end} GROUP BY DAY_ONLY(${column}) ORDER BY DAY_ONLY(${column})";
-  private static final String DAY_FORMAT = "yyyy-MM-dd";
+  private static final String ENABLE_DYNAMIC_PROBING = "salesforce.enableDynamicProbing";
+  private static final String DYNAMIC_PROBING_LIMIT = "salesforce.dynamicProbingLimit";
+  private static final int DEFAULT_DYNAMIC_PROBING_LIMIT = 1000;
+  private static final String MIN_TARGET_PARTITION_SIZE = "salesforce.minTargetPartitionSize";
+  private static final int DEFAULT_MIN_TARGET_PARTITION_SIZE = 250000;
+  // this is used to generate histogram buckets smaller than the target partition size to allow for more even
+  // packing of the generated partitions
+  private static final String PROBE_TARGET_RATIO = "salesforce.probeTargetRatio";
+  private static final double DEFAULT_PROBE_TARGET_RATIO = 0.60;
+  private static final int MIN_SPLIT_TIME_MILLIS = 1000;
+
+  private static final String DAY_PARTITION_QUERY_TEMPLATE =
+      "SELECT count(${column}) cnt, DAY_ONLY(${column}) time FROM ${table} " + "WHERE ${column} ${greater} ${start}"
+          + " AND ${column} ${less} ${end} GROUP BY DAY_ONLY(${column}) ORDER BY DAY_ONLY(${column})";
+  private static final String PROBE_PARTITION_QUERY_TEMPLATE = "SELECT count(${column}) cnt FROM ${table} "
+      + "WHERE ${column} ${greater} ${start} AND ${column} ${less} ${end}";
+
+  private static final String SECONDS_FORMAT = "yyyy-MM-dd-HH:mm:ss";
+  private static final String ZERO_TIME_SUFFIX = "-00:00:00";
 
   private static final Gson GSON = new Gson();
 
@@ -103,25 +120,28 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
 
     int maxPartitions = state.getPropAsInt(ConfigurationKeys.SOURCE_MAX_NUMBER_OF_PARTITIONS,
         ConfigurationKeys.DEFAULT_MAX_NUMBER_OF_PARTITIONS);
+    int minTargetPartitionSize = state.getPropAsInt(MIN_TARGET_PARTITION_SIZE, DEFAULT_MIN_TARGET_PARTITION_SIZE);
 
     // Only support time related watermark
-    if (watermarkType == WatermarkType.SIMPLE || Strings.isNullOrEmpty(watermarkColumn)
-        || !state.getPropAsBoolean(ENABLE_DYNAMIC_PARTITIONING) || maxPartitions <= 1) {
+    if (watermarkType == WatermarkType.SIMPLE || Strings.isNullOrEmpty(watermarkColumn) || !state.getPropAsBoolean(
+        ENABLE_DYNAMIC_PARTITIONING) || maxPartitions <= 1) {
       return super.generateWorkUnits(sourceEntity, state, previousWatermark);
     }
 
     Partition partition = new Partitioner(state).getGlobalPartition(previousWatermark);
     Histogram histogram = getHistogram(sourceEntity.getSourceEntityName(), watermarkColumn, state, partition);
 
-    String specifiedPartitions = generateSpecifiedPartitions(histogram, maxPartitions, partition.getHighWatermark());
+    String specifiedPartitions = generateSpecifiedPartitions(histogram, minTargetPartitionSize, maxPartitions,
+        partition.getLowWatermark(), partition.getHighWatermark());
     state.setProp(Partitioner.HAS_USER_SPECIFIED_PARTITIONS, true);
     state.setProp(Partitioner.USER_SPECIFIED_PARTITIONS, specifiedPartitions);
 
     return super.generateWorkUnits(sourceEntity, state, previousWatermark);
   }
 
-  String generateSpecifiedPartitions(Histogram histogram, int maxPartitions, long expectedHighWatermark) {
-    long interval = DoubleMath.roundToLong((double) histogram.totalRecordCount / maxPartitions, RoundingMode.CEILING);
+  String generateSpecifiedPartitions(Histogram histogram, int minTargetPartitionSize, int maxPartitions, long lowWatermark,
+      long expectedHighWatermark) {
+    int interval = computeTargetPartitionSize(histogram, minTargetPartitionSize, maxPartitions);
     int totalGroups = histogram.getGroups().size();
 
     log.info("Histogram total record count: " + histogram.totalRecordCount);
@@ -136,11 +156,12 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
     int count = 0;
     HistogramGroup group;
     Iterator<HistogramGroup> it = groups.iterator();
+
     while (it.hasNext()) {
       group = it.next();
       if (count == 0) {
         // Add a new partition point;
-        partitionPoints.add(Utils.toDateTimeFormat(group.getKey(), DAY_FORMAT, Partitioner.WATERMARKTIMEFORMAT));
+        partitionPoints.add(Utils.toDateTimeFormat(group.getKey(), SECONDS_FORMAT, Partitioner.WATERMARKTIMEFORMAT));
       }
 
       // Move the candidate to a new bucket if the attempted total is 2x of interval
@@ -148,7 +169,7 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
         // Summarize current group
         statistics.addValue(count);
         // A step-in start
-        partitionPoints.add(Utils.toDateTimeFormat(group.getKey(), DAY_FORMAT, Partitioner.WATERMARKTIMEFORMAT));
+        partitionPoints.add(Utils.toDateTimeFormat(group.getKey(), SECONDS_FORMAT, Partitioner.WATERMARKTIMEFORMAT));
         count = group.count;
       } else {
         // Add group into current partition
@@ -163,6 +184,10 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
       }
     }
 
+    if (partitionPoints.isEmpty()) {
+      throw new RuntimeException("Unexpected empty partition list");
+    }
+
     // If the last group is used as the last partition point
     if (count == 0) {
       // Exchange the last partition point with global high watermark
@@ -182,7 +207,166 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
     return specifiedPartitions;
   }
 
-  private Histogram getHistogram(String entity, String watermarkColumn, SourceState state,
+  /**
+   * Compute the target partition size.
+   */
+  private int computeTargetPartitionSize(Histogram histogram, int minTargetPartitionSize, int maxPartitions) {
+     return Math.max(minTargetPartitionSize,
+        DoubleMath.roundToInt((double) histogram.totalRecordCount / maxPartitions, RoundingMode.CEILING));
+  }
+
+  /**
+   * Get a {@link JsonArray} containing the query results
+   */
+  private JsonArray getRecordsForQuery(SalesforceConnector connector, String query) {
+    try {
+      String soqlQuery = SalesforceExtractor.getSoqlUrl(query);
+      List<Command> commands = RestApiConnector.constructGetCommand(connector.getFullUri(soqlQuery));
+      CommandOutput<?, ?> response = connector.getResponse(commands);
+
+      String output;
+      Iterator<String> itr = (Iterator<String>) response.getResults().values().iterator();
+      if (itr.hasNext()) {
+        output = itr.next();
+      } else {
+        throw new DataRecordException("Failed to get data from salesforce; REST response has no output");
+      }
+
+      return GSON.fromJson(output, JsonObject.class).getAsJsonArray("records");
+    } catch (RestApiClientException | RestApiProcessingException | DataRecordException e) {
+      throw new RuntimeException("Fail to get data from salesforce", e);
+    }
+  }
+
+  /**
+   * Get the row count for a time range
+   */
+  private int getCountForRange(TableCountProbingContext probingContext, StrSubstitutor sub,
+      Map<String, String> subValues, long startTime, long endTime) {
+    String startTimeStr = Utils.dateToString(new Date(startTime), SalesforceExtractor.SALESFORCE_TIMESTAMP_FORMAT);
+    String endTimeStr = Utils.dateToString(new Date(endTime), SalesforceExtractor.SALESFORCE_TIMESTAMP_FORMAT);
+
+    subValues.put("start", startTimeStr);
+    subValues.put("end", endTimeStr);
+
+    String query = sub.replace(PROBE_PARTITION_QUERY_TEMPLATE);
+
+    log.debug("Count query: " + query);
+    probingContext.probeCount++;
+
+    JsonArray records = getRecordsForQuery(probingContext.connector, query);
+    Iterator<JsonElement> elements = records.iterator();
+    JsonObject element = elements.next().getAsJsonObject();
+
+    return element.get("cnt").getAsInt();
+  }
+
+  /**
+   * Split a histogram bucket along the midpoint if it is larger than the bucket size limit.
+   */
+  private void getHistogramRecursively(TableCountProbingContext probingContext, Histogram histogram, StrSubstitutor sub,
+      Map<String, String> values, int count, long startEpoch, long endEpoch) {
+    long midpointEpoch = startEpoch + (endEpoch - startEpoch) / 2;
+
+    // don't split further if small, above the probe limit, or less than 1 second difference between the midpoint and start
+    if (count <= probingContext.bucketSizeLimit
+        || probingContext.probeCount > probingContext.probeLimit
+        || (midpointEpoch - startEpoch < MIN_SPLIT_TIME_MILLIS)) {
+      histogram.add(new HistogramGroup(Utils.epochToDate(startEpoch, SECONDS_FORMAT), count));
+      return;
+    }
+
+    int countLeft = getCountForRange(probingContext, sub, values, startEpoch, midpointEpoch);
+
+    getHistogramRecursively(probingContext, histogram, sub, values, countLeft, startEpoch, midpointEpoch);
+    log.debug("Count {} for left partition {} to {}", countLeft, startEpoch, midpointEpoch);
+
+    int countRight = count - countLeft;
+
+    getHistogramRecursively(probingContext, histogram, sub, values, countRight, midpointEpoch, endEpoch);
+    log.debug("Count {} for right partition {} to {}", countRight, midpointEpoch, endEpoch);
+  }
+
+  /**
+   * Get a histogram for the time range by probing to break down large buckets. Use count instead of
+   * querying if it is non-negative.
+   */
+  private Histogram getHistogramByProbing(TableCountProbingContext probingContext, int count, long startEpoch,
+      long endEpoch) {
+    Histogram histogram = new Histogram();
+
+    Map<String, String> values = new HashMap<>();
+    values.put("table", probingContext.entity);
+    values.put("column", probingContext.watermarkColumn);
+    values.put("greater", ">=");
+    values.put("less", "<");
+    StrSubstitutor sub = new StrSubstitutor(values);
+
+    getHistogramRecursively(probingContext, histogram, sub, values, count, startEpoch, endEpoch);
+
+    return histogram;
+  }
+
+  /**
+   * Refine the histogram by probing to split large buckets
+   * @return the refined histogram
+   */
+  private Histogram getRefinedHistogram(SalesforceConnector connector, String entity, String watermarkColumn,
+      SourceState state, Partition partition, Histogram histogram) {
+    final int maxPartitions = state.getPropAsInt(ConfigurationKeys.SOURCE_MAX_NUMBER_OF_PARTITIONS,
+        ConfigurationKeys.DEFAULT_MAX_NUMBER_OF_PARTITIONS);
+    final int probeLimit = state.getPropAsInt(DYNAMIC_PROBING_LIMIT, DEFAULT_DYNAMIC_PROBING_LIMIT);
+    final int minTargetPartitionSize = state.getPropAsInt(MIN_TARGET_PARTITION_SIZE, DEFAULT_MIN_TARGET_PARTITION_SIZE);
+    final Histogram outputHistogram = new Histogram();
+    final double probeTargetRatio = state.getPropAsDouble(PROBE_TARGET_RATIO, DEFAULT_PROBE_TARGET_RATIO);
+    final int bucketSizeLimit =
+        (int) (probeTargetRatio * computeTargetPartitionSize(histogram, minTargetPartitionSize, maxPartitions));
+
+    log.info("Refining histogram with bucket size limit {}.", bucketSizeLimit);
+
+    HistogramGroup currentGroup;
+    HistogramGroup nextGroup;
+    final TableCountProbingContext probingContext =
+        new TableCountProbingContext(connector, entity, watermarkColumn, bucketSizeLimit, probeLimit);
+
+    if (histogram.getGroups().isEmpty()) {
+      return outputHistogram;
+    }
+
+    // make a copy of the histogram list and add a dummy entry at the end to avoid special processing of the last group
+    List<HistogramGroup> list = new ArrayList(histogram.getGroups());
+    Date hwmDate = Utils.toDate(partition.getLowWatermark(), Partitioner.WATERMARKTIMEFORMAT);
+    list.add(new HistogramGroup(Utils.epochToDate(hwmDate.getTime(), SECONDS_FORMAT), 0));
+
+    for (int i = 0; i < list.size() - 1; i++) {
+      currentGroup = list.get(i);
+      nextGroup = list.get(i + 1);
+
+      // split the group if it is larger than the bucket size limit
+      if (currentGroup.count > bucketSizeLimit) {
+        long startEpoch = Utils.toDate(currentGroup.getKey(), SECONDS_FORMAT).getTime();
+        long endEpoch = Utils.toDate(nextGroup.getKey(), SECONDS_FORMAT).getTime();
+
+        outputHistogram.add(getHistogramByProbing(probingContext, currentGroup.count, startEpoch, endEpoch));
+      } else {
+        outputHistogram.add(currentGroup);
+      }
+    }
+
+    log.info("Executed {} probes for refining the histogram.", probingContext.probeCount);
+
+    // if the probe limit has been reached then print a warning
+    if (probingContext.probeCount >= probingContext.probeLimit) {
+      log.warn("Reached the probe limit");
+    }
+
+    return outputHistogram;
+  }
+
+  /**
+   * Get a histogram with day granularity buckets.
+   */
+  private Histogram getHistogramByDayBucketing(SalesforceConnector connector, String entity, String watermarkColumn,
       Partition partition) {
     Histogram histogram = new Histogram();
 
@@ -202,17 +386,7 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
     values.put("column", watermarkColumn);
     StrSubstitutor sub = new StrSubstitutor(values);
 
-    SalesforceConnector connector = new SalesforceConnector(state);
-    try {
-      if (!connector.connect()) {
-        throw new RuntimeException("Failed to connect.");
-      }
-    } catch (RestApiConnectionException e) {
-      throw new RuntimeException("Failed to connect.", e);
-    }
-
     for (int year = startYear; year <= endYear; year++) {
-
       if (year == startYear) {
         values.put("start", lowWatermarkDate);
         values.put("greater", partition.isLowWatermarkInclusive() ? ">=" : ">");
@@ -232,14 +406,39 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
       String query = sub.replace(DAY_PARTITION_QUERY_TEMPLATE);
       log.info("Histogram query: " + query);
 
-      try {
-        String soqlQuery = SalesforceExtractor.getSoqlUrl(query);
-        List<Command> commands = RestApiConnector.constructGetCommand(connector.getFullUri(soqlQuery));
-        CommandOutput<?, ?> response = connector.getResponse(commands);
-        histogram.add(parseHistogram(response));
-      } catch (RestApiClientException | RestApiProcessingException | DataRecordException e) {
-        throw new RuntimeException("Fail to get data of year " + year + " from salesforce", e);
+      histogram.add(parseDayBucketingHistogram(getRecordsForQuery(connector, query)));
+    }
+
+    return histogram;
+  }
+
+  /**
+   * Generate the histogram
+   */
+  private Histogram getHistogram(String entity, String watermarkColumn, SourceState state,
+      Partition partition) {
+    SalesforceConnector connector = new SalesforceConnector(state);
+
+    try {
+      if (!connector.connect()) {
+        throw new RuntimeException("Failed to connect.");
       }
+    } catch (RestApiConnectionException e) {
+      throw new RuntimeException("Failed to connect.", e);
+    }
+
+    Histogram histogram = getHistogramByDayBucketing(connector, entity, watermarkColumn, partition);
+
+    // exchange the first histogram group key with the global low watermark to ensure that the low watermark is captured
+    // in the range of generated partitions
+    HistogramGroup firstGroup = histogram.getGroups().get(0);
+    Date lwmDate = Utils.toDate(partition.getLowWatermark(), Partitioner.WATERMARKTIMEFORMAT);
+    histogram.getGroups().set(0, new HistogramGroup(Utils.epochToDate(lwmDate.getTime(), SECONDS_FORMAT),
+        firstGroup.getCount()));
+
+    // refine the histogram
+    if (state.getPropAsBoolean(ENABLE_DYNAMIC_PROBING)) {
+      histogram = getRefinedHistogram(connector, entity, watermarkColumn, state, partition, histogram);
     }
 
     return histogram;
@@ -252,23 +451,23 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
     return Utils.dateToString(calendar.getTime(), SalesforceExtractor.SALESFORCE_TIMESTAMP_FORMAT);
   }
 
-  private Histogram parseHistogram(CommandOutput<?, ?> response) throws DataRecordException {
-    log.info("Parse histogram");
+  /**
+   * Parse the query results into a {@link Histogram}
+   */
+  private Histogram parseDayBucketingHistogram(JsonArray records) {
+    log.info("Parse day-based histogram");
+
     Histogram histogram = new Histogram();
 
-    String output;
-    Iterator<String> itr = (Iterator<String>) response.getResults().values().iterator();
-    if (itr.hasNext()) {
-      output = itr.next();
-    } else {
-      throw new DataRecordException("Failed to get data from salesforce; REST response has no output");
-    }
-    JsonArray records = GSON.fromJson(output, JsonObject.class).getAsJsonArray("records");
     Iterator<JsonElement> elements = records.iterator();
     JsonObject element;
+
     while (elements.hasNext()) {
       element = elements.next().getAsJsonObject();
-      histogram.add(new HistogramGroup(element.get("time").getAsString(), element.get("cnt").getAsInt()));
+      String time = element.get("time").getAsString() + ZERO_TIME_SUFFIX;
+      int count = element.get("cnt").getAsInt();
+
+      histogram.add(new HistogramGroup(time, count));
     }
 
     return histogram;
@@ -354,4 +553,17 @@ public class SalesforceSource extends QueryBasedSource<JsonArray, JsonElement> {
     return result;
   }
 
+  /**
+   * Context for probing the table for row counts of a time range
+   */
+  @RequiredArgsConstructor
+  private static class TableCountProbingContext {
+    private final SalesforceConnector connector;
+    private final String entity;
+    private final String watermarkColumn;
+    private final int bucketSizeLimit;
+    private final int probeLimit;
+
+    private int probeCount = 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/626d312a/gobblin-salesforce/src/test/java/org/apache/gobblin/salesforce/SalesforceSourceTest.java
----------------------------------------------------------------------
diff --git a/gobblin-salesforce/src/test/java/org/apache/gobblin/salesforce/SalesforceSourceTest.java b/gobblin-salesforce/src/test/java/org/apache/gobblin/salesforce/SalesforceSourceTest.java
index 394d0fb..8e45c6b 100644
--- a/gobblin-salesforce/src/test/java/org/apache/gobblin/salesforce/SalesforceSourceTest.java
+++ b/gobblin-salesforce/src/test/java/org/apache/gobblin/salesforce/SalesforceSourceTest.java
@@ -25,17 +25,18 @@ public class SalesforceSourceTest {
   void testGenerateSpecifiedPartition() {
     SalesforceSource.Histogram histogram = new SalesforceSource.Histogram();
     for (String group: HISTOGRAM.split(", ")) {
-      String[] groupInfo = group.split(":");
+      String[] groupInfo = group.split("::");
       histogram.add(new SalesforceSource.HistogramGroup(groupInfo[0], Integer.parseInt(groupInfo[1])));
     }
     SalesforceSource source = new SalesforceSource();
 
     long expectedHighWatermark = 20170407152123L;
+    long lowWatermark = 20140213000000L;
     int maxPartitions = 5;
     String expectedPartitions = "20140213000000,20170224000000,20170228000000,20170301000000,20170407000000,20170407152123";
-    String actualPartitions = source.generateSpecifiedPartitions(histogram, maxPartitions, expectedHighWatermark);
+    String actualPartitions = source.generateSpecifiedPartitions(histogram, 1, maxPartitions, lowWatermark, expectedHighWatermark);
     Assert.assertEquals(actualPartitions, expectedPartitions);
   }
 
-  static final String HISTOGRAM = "2014-02-13:3, 2014-04-15:1, 2014-05-06:624, 2014-05-07:1497, 2014-05-08:10, 2014-05-18:3, 2014-05-19:2, 2014-05-20:1, 2014-05-21:8, 2014-05-26:2, 2014-05-28:1, 2014-05-31:1, 2014-06-02:1, 2014-06-03:1, 2014-06-04:1, 2014-06-10:2, 2014-06-12:1, 2014-06-23:1, 2014-06-24:1, 2014-06-26:32, 2014-06-27:40, 2014-06-30:2, 2014-07-01:2, 2014-07-02:1, 2014-07-07:1, 2014-07-08:2, 2014-07-09:2, 2014-07-10:3, 2014-07-11:5, 2014-07-14:1, 2014-07-15:2, 2014-07-16:8, 2014-07-17:5, 2014-07-18:2, 2014-07-21:1, 2014-07-22:3, 2014-07-23:3, 2014-07-24:1, 2014-07-25:1, 2014-07-26:103, 2014-07-28:1, 2014-07-29:1, 2014-07-30:3, 2014-08-01:1, 2014-08-06:3, 2014-08-18:3, 2014-08-19:2, 2014-08-21:1, 2014-08-25:1, 2014-08-26:1, 2014-08-27:2, 2014-08-28:2, 2014-08-29:1, 2014-09-03:2, 2014-09-04:1, 2014-09-05:9, 2014-09-09:1, 2014-09-10:1, 2014-09-11:4, 2014-09-12:1, 2014-09-15:1, 2014-09-16:7, 2014-09-18:3, 2014-09-19:2, 2014-09-21:1, 2014-09-24:3, 2014-09-26:1, 2014-09-29:3, 
 2014-10-03:1, 2014-10-06:1, 2014-10-07:1, 2014-10-09:1, 2014-10-16:1, 2014-10-23:1, 2014-10-24:1, 2014-10-28:1, 2014-11-03:1, 2014-11-05:2, 2014-11-07:1, 2014-11-10:5, 2014-11-12:1, 2014-11-13:4, 2014-11-18:1, 2014-11-24:1, 2014-11-25:1, 2014-11-26:2, 2014-11-27:2, 2014-11-28:1, 2014-12-01:2, 2014-12-02:3, 2014-12-03:5, 2014-12-04:1, 2014-12-08:1, 2014-12-09:3, 2014-12-12:3, 2014-12-14:1, 2014-12-15:4, 2014-12-16:1, 2014-12-17:2, 2014-12-19:2, 2014-12-22:1, 2014-12-23:3, 2014-12-24:1, 2014-12-30:1, 2014-12-31:1, 2015-01-02:1, 2015-01-04:1, 2015-01-05:1, 2015-01-06:1, 2015-01-07:1, 2015-01-08:2, 2015-01-09:2, 2015-01-14:1, 2015-01-15:2, 2015-01-16:15685, 2015-01-19:1, 2015-01-21:1, 2015-01-22:1, 2015-01-27:2, 2015-01-28:4, 2015-01-29:1, 2015-01-30:1, 2015-02-01:1, 2015-02-02:4, 2015-02-05:11, 2015-02-06:3, 2015-02-09:2, 2015-02-10:1, 2015-02-11:2, 2015-02-12:15, 2015-02-13:4, 2015-02-14:3, 2015-02-15:2, 2015-02-16:2, 2015-02-18:8, 2015-02-19:2, 2015-02-20:4, 2015-02-21:1, 2015-02-22:
 4, 2015-02-23:1, 2015-02-24:4, 2015-02-25:1, 2015-02-26:3, 2015-02-27:3, 2015-03-02:5, 2015-03-03:5, 2015-03-04:21, 2015-03-05:2, 2015-03-06:5, 2015-03-07:5, 2015-03-09:9, 2015-03-10:2050, 2015-03-11:13, 2015-03-12:3035, 2015-03-13:1, 2015-03-16:10, 2015-03-17:2, 2015-03-18:1, 2015-03-19:10, 2015-03-20:4, 2015-03-23:281, 2015-03-24:60, 2015-03-25:30, 2015-03-26:7, 2015-03-27:10, 2015-03-29:1, 2015-03-30:1, 2015-03-31:3, 2015-04-01:5, 2015-04-02:2, 2015-04-03:4, 2015-04-05:3, 2015-04-06:5, 2015-04-07:16, 2015-04-08:6, 2015-04-09:8, 2015-04-10:2, 2015-04-11:1, 2015-04-12:1, 2015-04-13:2, 2015-04-14:1, 2015-04-15:3, 2015-04-16:3, 2015-04-17:1, 2015-04-18:2, 2015-04-20:10, 2015-04-21:2, 2015-04-22:20, 2015-04-23:49, 2015-04-24:1, 2015-04-25:3, 2015-04-27:8, 2015-04-28:115, 2015-04-29:120, 2015-04-30:397, 2015-05-01:4, 2015-05-04:605, 2015-05-05:3, 2015-05-06:7, 2015-05-07:14, 2015-05-08:1, 2015-05-09:2, 2015-05-11:5, 2015-05-12:5, 2015-05-13:2, 2015-05-14:3, 2015-05-15:2, 2015-05-18:2, 
 2015-05-19:3, 2015-05-21:4, 2015-05-22:4, 2015-05-25:1, 2015-05-26:2, 2015-05-27:2, 2015-05-28:3, 2015-05-29:20, 2015-05-30:1, 2015-06-01:1, 2015-06-02:11, 2015-06-03:2, 2015-06-04:2, 2015-06-08:3, 2015-06-09:9, 2015-06-10:3, 2015-06-11:3, 2015-06-13:2, 2015-06-15:11, 2015-06-16:7, 2015-06-17:2, 2015-06-18:1, 2015-06-19:1, 2015-06-22:4, 2015-06-23:1, 2015-06-24:12, 2015-06-25:13, 2015-06-26:9, 2015-06-28:1, 2015-06-29:655, 2015-06-30:5, 2015-07-01:30, 2015-07-02:7, 2015-07-03:3, 2015-07-04:1, 2015-07-05:1, 2015-07-06:10, 2015-07-07:18, 2015-07-08:2, 2015-07-09:17, 2015-07-10:5, 2015-07-11:1, 2015-07-13:6, 2015-07-14:9, 2015-07-15:3, 2015-07-16:21, 2015-07-17:3, 2015-07-18:1, 2015-07-20:92, 2015-07-22:4, 2015-07-23:5, 2015-07-24:5, 2015-07-25:1, 2015-07-27:7, 2015-07-28:19, 2015-07-30:10, 2015-07-31:6, 2015-08-01:1, 2015-08-03:1, 2015-08-04:4, 2015-08-05:17, 2015-08-06:4, 2015-08-07:3, 2015-08-08:2, 2015-08-10:6, 2015-08-11:2, 2015-08-12:4, 2015-08-13:2, 2015-08-14:1, 2015-08-16:3, 2
 015-08-17:4, 2015-08-18:2, 2015-08-19:4, 2015-08-20:216, 2015-08-21:6, 2015-08-24:5, 2015-08-25:4, 2015-08-26:384, 2015-08-27:3, 2015-08-28:2, 2015-08-31:5, 2015-09-01:7, 2015-09-02:2, 2015-09-03:9, 2015-09-06:1, 2015-09-07:2, 2015-09-08:25, 2015-09-09:4, 2015-09-10:5, 2015-09-11:3, 2015-09-12:1, 2015-09-13:1, 2015-09-14:4, 2015-09-15:45, 2015-09-16:2, 2015-09-17:2, 2015-09-18:2, 2015-09-21:205, 2015-09-22:88, 2015-09-23:23, 2015-09-24:37, 2015-09-25:7, 2015-09-28:5, 2015-09-29:35, 2015-09-30:24, 2015-10-01:16, 2015-10-02:31, 2015-10-04:1, 2015-10-05:5, 2015-10-06:1, 2015-10-07:10, 2015-10-08:9, 2015-10-09:8, 2015-10-10:219, 2015-10-11:77, 2015-10-12:191, 2015-10-13:9, 2015-10-14:23, 2015-10-15:103, 2015-10-16:52, 2015-10-17:1, 2015-10-18:5, 2015-10-19:4, 2015-10-20:822, 2015-10-21:34, 2015-10-22:41, 2015-10-23:2045, 2015-10-24:1, 2015-10-25:2, 2015-10-26:7, 2015-10-27:19, 2015-10-28:17, 2015-10-29:14, 2015-10-30:12, 2015-11-02:4, 2015-11-03:7, 2015-11-04:11, 2015-11-05:6, 2015-11-0
 6:8, 2015-11-07:2, 2015-11-08:1, 2015-11-09:10, 2015-11-10:10, 2015-11-11:10, 2015-11-12:5, 2015-11-13:6, 2015-11-15:3, 2015-11-16:8, 2015-11-17:4, 2015-11-18:8, 2015-11-19:458, 2015-11-20:390, 2015-11-22:1, 2015-11-23:12, 2015-11-24:13, 2015-11-25:542, 2015-11-26:2, 2015-11-27:1, 2015-11-30:4, 2015-12-01:3, 2015-12-02:4, 2015-12-03:8, 2015-12-04:6, 2015-12-05:1, 2015-12-06:1, 2015-12-07:3, 2015-12-08:18, 2015-12-09:3, 2015-12-10:22, 2015-12-11:4, 2015-12-14:13, 2015-12-15:8, 2015-12-16:8, 2015-12-17:5, 2015-12-18:9, 2015-12-20:2, 2015-12-21:7, 2015-12-22:4, 2015-12-23:15, 2015-12-24:2, 2015-12-28:2, 2015-12-31:1, 2016-01-01:10, 2016-01-02:2, 2016-01-03:1, 2016-01-04:15, 2016-01-05:9, 2016-01-06:19, 2016-01-07:30, 2016-01-08:9711, 2016-01-09:9, 2016-01-10:9, 2016-01-11:20, 2016-01-12:14, 2016-01-13:3084, 2016-01-14:17, 2016-01-15:9, 2016-01-16:1, 2016-01-17:1, 2016-01-18:2, 2016-01-19:9, 2016-01-20:12, 2016-01-21:15, 2016-01-22:9, 2016-01-23:6, 2016-01-24:2, 2016-01-25:3, 2016-01-26
 :1, 2016-01-27:3, 2016-01-28:7, 2016-01-29:4, 2016-01-30:7, 2016-01-31:3, 2016-02-01:17, 2016-02-02:28, 2016-02-03:20, 2016-02-04:20, 2016-02-05:40, 2016-02-06:3, 2016-02-07:9, 2016-02-08:28, 2016-02-09:33, 2016-02-10:100, 2016-02-11:52, 2016-02-12:103, 2016-02-13:5, 2016-02-14:5, 2016-02-15:7, 2016-02-16:23, 2016-02-17:210, 2016-02-18:29, 2016-02-19:24, 2016-02-20:5, 2016-02-21:29, 2016-02-22:26, 2016-02-23:12, 2016-02-24:19, 2016-02-25:46, 2016-02-26:23, 2016-02-27:7, 2016-02-28:1, 2016-02-29:45, 2016-03-01:48, 2016-03-02:49, 2016-03-03:34, 2016-03-04:61, 2016-03-05:5, 2016-03-06:15, 2016-03-07:22, 2016-03-08:3649, 2016-03-09:49, 2016-03-10:39, 2016-03-11:67, 2016-03-12:3, 2016-03-13:1, 2016-03-14:31, 2016-03-15:31, 2016-03-16:37, 2016-03-17:43, 2016-03-18:47, 2016-03-19:5, 2016-03-20:14, 2016-03-21:47, 2016-03-22:31, 2016-03-23:41, 2016-03-24:38, 2016-03-25:36, 2016-03-26:5, 2016-03-27:3, 2016-03-28:42, 2016-03-29:43, 2016-03-30:68, 2016-03-31:35, 2016-04-01:26, 2016-04-02:1, 201
 6-04-03:2, 2016-04-04:66, 2016-04-05:35, 2016-04-06:26, 2016-04-07:25, 2016-04-08:31, 2016-04-09:2, 2016-04-10:1, 2016-04-11:27, 2016-04-12:35, 2016-04-13:5, 2016-04-14:185, 2016-04-15:121, 2016-04-16:83, 2016-04-18:9, 2016-04-19:1, 2016-04-20:53, 2016-04-21:1, 2016-04-22:95, 2016-04-23:126, 2016-04-24:3, 2016-04-25:1, 2016-04-26:202, 2016-04-27:58, 2016-04-28:64, 2016-04-29:54, 2016-04-30:139, 2016-05-01:2, 2016-05-02:4, 2016-05-03:46, 2016-05-04:108, 2016-05-05:34, 2016-05-06:24, 2016-05-07:7, 2016-05-08:3, 2016-05-09:44, 2016-05-10:28, 2016-05-11:44, 2016-05-12:58, 2016-05-13:40, 2016-05-14:2, 2016-05-15:1, 2016-05-16:29, 2016-05-17:16, 2016-05-18:31, 2016-05-19:58, 2016-05-20:59, 2016-05-21:14, 2016-05-22:16, 2016-05-23:68, 2016-05-24:19, 2016-05-25:55, 2016-05-26:78, 2016-05-27:64, 2016-05-28:76, 2016-05-29:2, 2016-05-30:15, 2016-05-31:24, 2016-06-01:11, 2016-06-02:31, 2016-06-03:39, 2016-06-04:2, 2016-06-06:783, 2016-06-07:14, 2016-06-08:51, 2016-06-09:25, 2016-06-10:14, 2016-
 06-12:2, 2016-06-13:57, 2016-06-14:20, 2016-06-15:36, 2016-06-16:15, 2016-06-17:49, 2016-06-18:3, 2016-06-20:38, 2016-06-21:45, 2016-06-22:5215, 2016-06-23:4977, 2016-06-24:50, 2016-06-25:2, 2016-06-26:2, 2016-06-27:27, 2016-06-28:1102, 2016-06-29:59, 2016-06-30:38, 2016-07-01:34, 2016-07-02:1, 2016-07-03:3, 2016-07-04:69, 2016-07-05:26, 2016-07-06:5, 2016-07-07:5, 2016-07-08:26, 2016-07-09:58, 2016-07-10:3, 2016-07-11:13, 2016-07-12:1, 2016-07-13:68, 2016-07-14:73, 2016-07-15:69, 2016-07-16:65, 2016-07-17:4, 2016-07-18:1, 2016-07-19:14, 2016-07-20:60, 2016-07-21:328, 2016-07-22:41, 2016-07-23:105, 2016-07-24:16, 2016-07-25:1, 2016-07-26:37, 2016-07-27:72, 2016-07-28:59, 2016-07-29:53, 2016-07-30:241, 2016-07-31:1, 2016-08-01:10, 2016-08-02:2, 2016-08-03:45, 2016-08-04:44, 2016-08-05:76, 2016-08-06:96, 2016-08-07:8, 2016-08-08:129, 2016-08-09:14, 2016-08-10:50, 2016-08-11:47, 2016-08-12:56, 2016-08-13:95, 2016-08-14:6, 2016-08-15:20, 2016-08-16:11, 2016-08-17:5, 2016-08-18:94, 2016-
 08-19:51, 2016-08-20:110, 2016-08-21:2, 2016-08-22:30, 2016-08-23:3, 2016-08-24:50, 2016-08-25:55, 2016-08-26:51, 2016-08-27:103, 2016-08-28:5, 2016-08-29:6, 2016-08-31:58, 2016-09-01:72, 2016-09-02:94, 2016-09-03:107, 2016-09-04:9, 2016-09-05:1, 2016-09-06:10, 2016-09-07:58, 2016-09-08:82, 2016-09-09:44, 2016-09-10:162, 2016-09-11:5, 2016-09-12:5, 2016-09-13:1, 2016-09-14:64, 2016-09-15:62, 2016-09-16:55, 2016-09-17:106, 2016-09-18:17, 2016-09-19:6, 2016-09-20:4, 2016-09-21:68, 2016-09-22:69, 2016-09-23:62, 2016-09-24:101, 2016-09-25:11, 2016-09-26:13, 2016-09-27:2, 2016-09-28:94, 2016-09-29:42, 2016-09-30:47, 2016-10-01:109, 2016-10-02:13, 2016-10-03:2, 2016-10-04:43, 2016-10-05:38, 2016-10-06:41, 2016-10-07:31, 2016-10-08:64, 2016-10-09:7, 2016-10-10:13, 2016-10-11:2, 2016-10-12:47, 2016-10-13:49, 2016-10-14:114, 2016-10-15:96, 2016-10-16:9, 2016-10-17:3, 2016-10-18:3, 2016-10-19:60, 2016-10-20:9, 2016-10-21:160, 2016-10-22:53, 2016-10-23:2, 2016-10-24:4, 2016-10-25:4, 2016-10-26
 :69, 2016-10-27:67, 2016-10-28:4, 2016-10-29:182, 2016-10-31:8, 2016-11-01:72, 2016-11-02:68, 2016-11-03:53, 2016-11-04:54, 2016-11-05:94, 2016-11-06:2, 2016-11-07:24, 2016-11-08:14, 2016-11-09:83, 2016-11-10:71, 2016-11-11:74, 2016-11-12:129, 2016-11-13:3, 2016-11-14:4, 2016-11-16:55, 2016-11-17:54, 2016-11-18:77, 2016-11-19:119, 2016-11-20:4, 2016-11-21:14, 2016-11-22:3, 2016-11-23:52, 2016-11-24:33, 2016-11-25:45, 2016-11-26:66, 2016-11-27:4, 2016-11-28:3, 2016-11-29:1, 2016-11-30:60, 2016-12-01:78, 2016-12-02:84, 2016-12-03:66, 2016-12-04:7, 2016-12-05:38, 2016-12-06:53, 2016-12-07:147, 2016-12-08:472, 2016-12-09:112, 2016-12-10:80, 2016-12-11:8, 2016-12-12:9, 2016-12-14:42, 2016-12-16:76, 2016-12-17:105, 2016-12-18:5, 2016-12-19:8, 2016-12-21:63, 2016-12-22:31, 2016-12-23:56, 2016-12-24:51, 2016-12-25:9, 2016-12-26:1, 2016-12-28:3, 2016-12-30:7, 2016-12-31:4, 2017-01-02:1, 2017-01-04:10, 2017-01-05:57, 2017-01-06:133, 2017-01-07:99, 2017-01-08:5, 2017-01-09:5, 2017-01-10:14, 20
 17-01-11:61, 2017-01-12:39, 2017-01-13:55, 2017-01-14:107, 2017-01-15:5, 2017-01-16:3, 2017-01-18:39, 2017-01-19:48, 2017-01-20:38, 2017-01-21:73, 2017-01-22:6, 2017-01-23:10, 2017-01-25:1, 2017-01-26:104, 2017-01-27:1, 2017-01-28:117, 2017-01-29:3, 2017-01-30:4, 2017-01-31:78, 2017-02-01:60, 2017-02-02:49, 2017-02-03:46, 2017-02-04:113, 2017-02-05:6, 2017-02-06:56, 2017-02-07:302, 2017-02-08:75, 2017-02-09:54, 2017-02-10:226, 2017-02-11:1039, 2017-02-13:29515, 2017-02-14:602, 2017-02-15:1099, 2017-02-16:97, 2017-02-17:1608, 2017-02-18:1258, 2017-02-19:109, 2017-02-20:116, 2017-02-21:336, 2017-02-22:664, 2017-02-23:154529, 2017-02-24:40521, 2017-02-25:2100, 2017-02-26:131, 2017-02-27:415645, 2017-02-28:276943, 2017-03-01:3306, 2017-03-02:3552, 2017-03-03:4227, 2017-03-04:6963, 2017-03-05:440, 2017-03-06:3365, 2017-03-07:389, 2017-03-08:3870, 2017-03-09:3867, 2017-03-10:4218, 2017-03-11:7977, 2017-03-12:391, 2017-03-13:27355, 2017-03-14:34284, 2017-03-15:14698, 2017-03-16:2926, 2017-
 03-17:5338, 2017-03-18:7752, 2017-03-19:510, 2017-03-20:7459, 2017-03-21:440, 2017-03-22:6040, 2017-03-23:6313, 2017-03-24:5792, 2017-03-25:9397, 2017-03-26:613, 2017-03-27:1365, 2017-03-28:7063, 2017-03-29:2110, 2017-03-30:8154, 2017-03-31:9777, 2017-04-01:16354, 2017-04-02:1288, 2017-04-03:1426, 2017-04-04:2666, 2017-04-05:13562, 2017-04-06:54698, 2017-04-07:22457";
+  static final String HISTOGRAM = "2014-02-13-00:00:00::3, 2014-04-15-00:00:00::1, 2014-05-06-00:00:00::624, 2014-05-07-00:00:00::1497, 2014-05-08-00:00:00::10, 2014-05-18-00:00:00::3, 2014-05-19-00:00:00::2, 2014-05-20-00:00:00::1, 2014-05-21-00:00:00::8, 2014-05-26-00:00:00::2, 2014-05-28-00:00:00::1, 2014-05-31-00:00:00::1, 2014-06-02-00:00:00::1, 2014-06-03-00:00:00::1, 2014-06-04-00:00:00::1, 2014-06-10-00:00:00::2, 2014-06-12-00:00:00::1, 2014-06-23-00:00:00::1, 2014-06-24-00:00:00::1, 2014-06-26-00:00:00::32, 2014-06-27-00:00:00::40, 2014-06-30-00:00:00::2, 2014-07-01-00:00:00::2, 2014-07-02-00:00:00::1, 2014-07-07-00:00:00::1, 2014-07-08-00:00:00::2, 2014-07-09-00:00:00::2, 2014-07-10-00:00:00::3, 2014-07-11-00:00:00::5, 2014-07-14-00:00:00::1, 2014-07-15-00:00:00::2, 2014-07-16-00:00:00::8, 2014-07-17-00:00:00::5, 2014-07-18-00:00:00::2, 2014-07-21-00:00:00::1, 2014-07-22-00:00:00::3, 2014-07-23-00:00:00::3, 2014-07-24-00:00:00::1, 2014-07-25-00:00:00::1, 2014-07-26-00:00:0
 0::103, 2014-07-28-00:00:00::1, 2014-07-29-00:00:00::1, 2014-07-30-00:00:00::3, 2014-08-01-00:00:00::1, 2014-08-06-00:00:00::3, 2014-08-18-00:00:00::3, 2014-08-19-00:00:00::2, 2014-08-21-00:00:00::1, 2014-08-25-00:00:00::1, 2014-08-26-00:00:00::1, 2014-08-27-00:00:00::2, 2014-08-28-00:00:00::2, 2014-08-29-00:00:00::1, 2014-09-03-00:00:00::2, 2014-09-04-00:00:00::1, 2014-09-05-00:00:00::9, 2014-09-09-00:00:00::1, 2014-09-10-00:00:00::1, 2014-09-11-00:00:00::4, 2014-09-12-00:00:00::1, 2014-09-15-00:00:00::1, 2014-09-16-00:00:00::7, 2014-09-18-00:00:00::3, 2014-09-19-00:00:00::2, 2014-09-21-00:00:00::1, 2014-09-24-00:00:00::3, 2014-09-26-00:00:00::1, 2014-09-29-00:00:00::3, 2014-10-03-00:00:00::1, 2014-10-06-00:00:00::1, 2014-10-07-00:00:00::1, 2014-10-09-00:00:00::1, 2014-10-16-00:00:00::1, 2014-10-23-00:00:00::1, 2014-10-24-00:00:00::1, 2014-10-28-00:00:00::1, 2014-11-03-00:00:00::1, 2014-11-05-00:00:00::2, 2014-11-07-00:00:00::1, 2014-11-10-00:00:00::5, 2014-11-12-00:00:00::1, 2014-
 11-13-00:00:00::4, 2014-11-18-00:00:00::1, 2014-11-24-00:00:00::1, 2014-11-25-00:00:00::1, 2014-11-26-00:00:00::2, 2014-11-27-00:00:00::2, 2014-11-28-00:00:00::1, 2014-12-01-00:00:00::2, 2014-12-02-00:00:00::3, 2014-12-03-00:00:00::5, 2014-12-04-00:00:00::1, 2014-12-08-00:00:00::1, 2014-12-09-00:00:00::3, 2014-12-12-00:00:00::3, 2014-12-14-00:00:00::1, 2014-12-15-00:00:00::4, 2014-12-16-00:00:00::1, 2014-12-17-00:00:00::2, 2014-12-19-00:00:00::2, 2014-12-22-00:00:00::1, 2014-12-23-00:00:00::3, 2014-12-24-00:00:00::1, 2014-12-30-00:00:00::1, 2014-12-31-00:00:00::1, 2015-01-02-00:00:00::1, 2015-01-04-00:00:00::1, 2015-01-05-00:00:00::1, 2015-01-06-00:00:00::1, 2015-01-07-00:00:00::1, 2015-01-08-00:00:00::2, 2015-01-09-00:00:00::2, 2015-01-14-00:00:00::1, 2015-01-15-00:00:00::2, 2015-01-16-00:00:00::15685, 2015-01-19-00:00:00::1, 2015-01-21-00:00:00::1, 2015-01-22-00:00:00::1, 2015-01-27-00:00:00::2, 2015-01-28-00:00:00::4, 2015-01-29-00:00:00::1, 2015-01-30-00:00:00::1, 2015-02-01-00:
 00:00::1, 2015-02-02-00:00:00::4, 2015-02-05-00:00:00::11, 2015-02-06-00:00:00::3, 2015-02-09-00:00:00::2, 2015-02-10-00:00:00::1, 2015-02-11-00:00:00::2, 2015-02-12-00:00:00::15, 2015-02-13-00:00:00::4, 2015-02-14-00:00:00::3, 2015-02-15-00:00:00::2, 2015-02-16-00:00:00::2, 2015-02-18-00:00:00::8, 2015-02-19-00:00:00::2, 2015-02-20-00:00:00::4, 2015-02-21-00:00:00::1, 2015-02-22-00:00:00::4, 2015-02-23-00:00:00::1, 2015-02-24-00:00:00::4, 2015-02-25-00:00:00::1, 2015-02-26-00:00:00::3, 2015-02-27-00:00:00::3, 2015-03-02-00:00:00::5, 2015-03-03-00:00:00::5, 2015-03-04-00:00:00::21, 2015-03-05-00:00:00::2, 2015-03-06-00:00:00::5, 2015-03-07-00:00:00::5, 2015-03-09-00:00:00::9, 2015-03-10-00:00:00::2050, 2015-03-11-00:00:00::13, 2015-03-12-00:00:00::3035, 2015-03-13-00:00:00::1, 2015-03-16-00:00:00::10, 2015-03-17-00:00:00::2, 2015-03-18-00:00:00::1, 2015-03-19-00:00:00::10, 2015-03-20-00:00:00::4, 2015-03-23-00:00:00::281, 2015-03-24-00:00:00::60, 2015-03-25-00:00:00::30, 2015-03-26-
 00:00:00::7, 2015-03-27-00:00:00::10, 2015-03-29-00:00:00::1, 2015-03-30-00:00:00::1, 2015-03-31-00:00:00::3, 2015-04-01-00:00:00::5, 2015-04-02-00:00:00::2, 2015-04-03-00:00:00::4, 2015-04-05-00:00:00::3, 2015-04-06-00:00:00::5, 2015-04-07-00:00:00::16, 2015-04-08-00:00:00::6, 2015-04-09-00:00:00::8, 2015-04-10-00:00:00::2, 2015-04-11-00:00:00::1, 2015-04-12-00:00:00::1, 2015-04-13-00:00:00::2, 2015-04-14-00:00:00::1, 2015-04-15-00:00:00::3, 2015-04-16-00:00:00::3, 2015-04-17-00:00:00::1, 2015-04-18-00:00:00::2, 2015-04-20-00:00:00::10, 2015-04-21-00:00:00::2, 2015-04-22-00:00:00::20, 2015-04-23-00:00:00::49, 2015-04-24-00:00:00::1, 2015-04-25-00:00:00::3, 2015-04-27-00:00:00::8, 2015-04-28-00:00:00::115, 2015-04-29-00:00:00::120, 2015-04-30-00:00:00::397, 2015-05-01-00:00:00::4, 2015-05-04-00:00:00::605, 2015-05-05-00:00:00::3, 2015-05-06-00:00:00::7, 2015-05-07-00:00:00::14, 2015-05-08-00:00:00::1, 2015-05-09-00:00:00::2, 2015-05-11-00:00:00::5, 2015-05-12-00:00:00::5, 2015-05-13
 -00:00:00::2, 2015-05-14-00:00:00::3, 2015-05-15-00:00:00::2, 2015-05-18-00:00:00::2, 2015-05-19-00:00:00::3, 2015-05-21-00:00:00::4, 2015-05-22-00:00:00::4, 2015-05-25-00:00:00::1, 2015-05-26-00:00:00::2, 2015-05-27-00:00:00::2, 2015-05-28-00:00:00::3, 2015-05-29-00:00:00::20, 2015-05-30-00:00:00::1, 2015-06-01-00:00:00::1, 2015-06-02-00:00:00::11, 2015-06-03-00:00:00::2, 2015-06-04-00:00:00::2, 2015-06-08-00:00:00::3, 2015-06-09-00:00:00::9, 2015-06-10-00:00:00::3, 2015-06-11-00:00:00::3, 2015-06-13-00:00:00::2, 2015-06-15-00:00:00::11, 2015-06-16-00:00:00::7, 2015-06-17-00:00:00::2, 2015-06-18-00:00:00::1, 2015-06-19-00:00:00::1, 2015-06-22-00:00:00::4, 2015-06-23-00:00:00::1, 2015-06-24-00:00:00::12, 2015-06-25-00:00:00::13, 2015-06-26-00:00:00::9, 2015-06-28-00:00:00::1, 2015-06-29-00:00:00::655, 2015-06-30-00:00:00::5, 2015-07-01-00:00:00::30, 2015-07-02-00:00:00::7, 2015-07-03-00:00:00::3, 2015-07-04-00:00:00::1, 2015-07-05-00:00:00::1, 2015-07-06-00:00:00::10, 2015-07-07-00:
 00:00::18, 2015-07-08-00:00:00::2, 2015-07-09-00:00:00::17, 2015-07-10-00:00:00::5, 2015-07-11-00:00:00::1, 2015-07-13-00:00:00::6, 2015-07-14-00:00:00::9, 2015-07-15-00:00:00::3, 2015-07-16-00:00:00::21, 2015-07-17-00:00:00::3, 2015-07-18-00:00:00::1, 2015-07-20-00:00:00::92, 2015-07-22-00:00:00::4, 2015-07-23-00:00:00::5, 2015-07-24-00:00:00::5, 2015-07-25-00:00:00::1, 2015-07-27-00:00:00::7, 2015-07-28-00:00:00::19, 2015-07-30-00:00:00::10, 2015-07-31-00:00:00::6, 2015-08-01-00:00:00::1, 2015-08-03-00:00:00::1, 2015-08-04-00:00:00::4, 2015-08-05-00:00:00::17, 2015-08-06-00:00:00::4, 2015-08-07-00:00:00::3, 2015-08-08-00:00:00::2, 2015-08-10-00:00:00::6, 2015-08-11-00:00:00::2, 2015-08-12-00:00:00::4, 2015-08-13-00:00:00::2, 2015-08-14-00:00:00::1, 2015-08-16-00:00:00::3, 2015-08-17-00:00:00::4, 2015-08-18-00:00:00::2, 2015-08-19-00:00:00::4, 2015-08-20-00:00:00::216, 2015-08-21-00:00:00::6, 2015-08-24-00:00:00::5, 2015-08-25-00:00:00::4, 2015-08-26-00:00:00::384, 2015-08-27-00:00
 :00::3, 2015-08-28-00:00:00::2, 2015-08-31-00:00:00::5, 2015-09-01-00:00:00::7, 2015-09-02-00:00:00::2, 2015-09-03-00:00:00::9, 2015-09-06-00:00:00::1, 2015-09-07-00:00:00::2, 2015-09-08-00:00:00::25, 2015-09-09-00:00:00::4, 2015-09-10-00:00:00::5, 2015-09-11-00:00:00::3, 2015-09-12-00:00:00::1, 2015-09-13-00:00:00::1, 2015-09-14-00:00:00::4, 2015-09-15-00:00:00::45, 2015-09-16-00:00:00::2, 2015-09-17-00:00:00::2, 2015-09-18-00:00:00::2, 2015-09-21-00:00:00::205, 2015-09-22-00:00:00::88, 2015-09-23-00:00:00::23, 2015-09-24-00:00:00::37, 2015-09-25-00:00:00::7, 2015-09-28-00:00:00::5, 2015-09-29-00:00:00::35, 2015-09-30-00:00:00::24, 2015-10-01-00:00:00::16, 2015-10-02-00:00:00::31, 2015-10-04-00:00:00::1, 2015-10-05-00:00:00::5, 2015-10-06-00:00:00::1, 2015-10-07-00:00:00::10, 2015-10-08-00:00:00::9, 2015-10-09-00:00:00::8, 2015-10-10-00:00:00::219, 2015-10-11-00:00:00::77, 2015-10-12-00:00:00::191, 2015-10-13-00:00:00::9, 2015-10-14-00:00:00::23, 2015-10-15-00:00:00::103, 2015-10-1
 6-00:00:00::52, 2015-10-17-00:00:00::1, 2015-10-18-00:00:00::5, 2015-10-19-00:00:00::4, 2015-10-20-00:00:00::822, 2015-10-21-00:00:00::34, 2015-10-22-00:00:00::41, 2015-10-23-00:00:00::2045, 2015-10-24-00:00:00::1, 2015-10-25-00:00:00::2, 2015-10-26-00:00:00::7, 2015-10-27-00:00:00::19, 2015-10-28-00:00:00::17, 2015-10-29-00:00:00::14, 2015-10-30-00:00:00::12, 2015-11-02-00:00:00::4, 2015-11-03-00:00:00::7, 2015-11-04-00:00:00::11, 2015-11-05-00:00:00::6, 2015-11-06-00:00:00::8, 2015-11-07-00:00:00::2, 2015-11-08-00:00:00::1, 2015-11-09-00:00:00::10, 2015-11-10-00:00:00::10, 2015-11-11-00:00:00::10, 2015-11-12-00:00:00::5, 2015-11-13-00:00:00::6, 2015-11-15-00:00:00::3, 2015-11-16-00:00:00::8, 2015-11-17-00:00:00::4, 2015-11-18-00:00:00::8, 2015-11-19-00:00:00::458, 2015-11-20-00:00:00::390, 2015-11-22-00:00:00::1, 2015-11-23-00:00:00::12, 2015-11-24-00:00:00::13, 2015-11-25-00:00:00::542, 2015-11-26-00:00:00::2, 2015-11-27-00:00:00::1, 2015-11-30-00:00:00::4, 2015-12-01-00:00:00::3
 , 2015-12-02-00:00:00::4, 2015-12-03-00:00:00::8, 2015-12-04-00:00:00::6, 2015-12-05-00:00:00::1, 2015-12-06-00:00:00::1, 2015-12-07-00:00:00::3, 2015-12-08-00:00:00::18, 2015-12-09-00:00:00::3, 2015-12-10-00:00:00::22, 2015-12-11-00:00:00::4, 2015-12-14-00:00:00::13, 2015-12-15-00:00:00::8, 2015-12-16-00:00:00::8, 2015-12-17-00:00:00::5, 2015-12-18-00:00:00::9, 2015-12-20-00:00:00::2, 2015-12-21-00:00:00::7, 2015-12-22-00:00:00::4, 2015-12-23-00:00:00::15, 2015-12-24-00:00:00::2, 2015-12-28-00:00:00::2, 2015-12-31-00:00:00::1, 2016-01-01-00:00:00::10, 2016-01-02-00:00:00::2, 2016-01-03-00:00:00::1, 2016-01-04-00:00:00::15, 2016-01-05-00:00:00::9, 2016-01-06-00:00:00::19, 2016-01-07-00:00:00::30, 2016-01-08-00:00:00::9711, 2016-01-09-00:00:00::9, 2016-01-10-00:00:00::9, 2016-01-11-00:00:00::20, 2016-01-12-00:00:00::14, 2016-01-13-00:00:00::3084, 2016-01-14-00:00:00::17, 2016-01-15-00:00:00::9, 2016-01-16-00:00:00::1, 2016-01-17-00:00:00::1, 2016-01-18-00:00:00::2, 2016-01-19-00:00:0
 0::9, 2016-01-20-00:00:00::12, 2016-01-21-00:00:00::15, 2016-01-22-00:00:00::9, 2016-01-23-00:00:00::6, 2016-01-24-00:00:00::2, 2016-01-25-00:00:00::3, 2016-01-26-00:00:00::1, 2016-01-27-00:00:00::3, 2016-01-28-00:00:00::7, 2016-01-29-00:00:00::4, 2016-01-30-00:00:00::7, 2016-01-31-00:00:00::3, 2016-02-01-00:00:00::17, 2016-02-02-00:00:00::28, 2016-02-03-00:00:00::20, 2016-02-04-00:00:00::20, 2016-02-05-00:00:00::40, 2016-02-06-00:00:00::3, 2016-02-07-00:00:00::9, 2016-02-08-00:00:00::28, 2016-02-09-00:00:00::33, 2016-02-10-00:00:00::100, 2016-02-11-00:00:00::52, 2016-02-12-00:00:00::103, 2016-02-13-00:00:00::5, 2016-02-14-00:00:00::5, 2016-02-15-00:00:00::7, 2016-02-16-00:00:00::23, 2016-02-17-00:00:00::210, 2016-02-18-00:00:00::29, 2016-02-19-00:00:00::24, 2016-02-20-00:00:00::5, 2016-02-21-00:00:00::29, 2016-02-22-00:00:00::26, 2016-02-23-00:00:00::12, 2016-02-24-00:00:00::19, 2016-02-25-00:00:00::46, 2016-02-26-00:00:00::23, 2016-02-27-00:00:00::7, 2016-02-28-00:00:00::1, 2016-0
 2-29-00:00:00::45, 2016-03-01-00:00:00::48, 2016-03-02-00:00:00::49, 2016-03-03-00:00:00::34, 2016-03-04-00:00:00::61, 2016-03-05-00:00:00::5, 2016-03-06-00:00:00::15, 2016-03-07-00:00:00::22, 2016-03-08-00:00:00::3649, 2016-03-09-00:00:00::49, 2016-03-10-00:00:00::39, 2016-03-11-00:00:00::67, 2016-03-12-00:00:00::3, 2016-03-13-00:00:00::1, 2016-03-14-00:00:00::31, 2016-03-15-00:00:00::31, 2016-03-16-00:00:00::37, 2016-03-17-00:00:00::43, 2016-03-18-00:00:00::47, 2016-03-19-00:00:00::5, 2016-03-20-00:00:00::14, 2016-03-21-00:00:00::47, 2016-03-22-00:00:00::31, 2016-03-23-00:00:00::41, 2016-03-24-00:00:00::38, 2016-03-25-00:00:00::36, 2016-03-26-00:00:00::5, 2016-03-27-00:00:00::3, 2016-03-28-00:00:00::42, 2016-03-29-00:00:00::43, 2016-03-30-00:00:00::68, 2016-03-31-00:00:00::35, 2016-04-01-00:00:00::26, 2016-04-02-00:00:00::1, 2016-04-03-00:00:00::2, 2016-04-04-00:00:00::66, 2016-04-05-00:00:00::35, 2016-04-06-00:00:00::26, 2016-04-07-00:00:00::25, 2016-04-08-00:00:00::31, 2016-04-0
 9-00:00:00::2, 2016-04-10-00:00:00::1, 2016-04-11-00:00:00::27, 2016-04-12-00:00:00::35, 2016-04-13-00:00:00::5, 2016-04-14-00:00:00::185, 2016-04-15-00:00:00::121, 2016-04-16-00:00:00::83, 2016-04-18-00:00:00::9, 2016-04-19-00:00:00::1, 2016-04-20-00:00:00::53, 2016-04-21-00:00:00::1, 2016-04-22-00:00:00::95, 2016-04-23-00:00:00::126, 2016-04-24-00:00:00::3, 2016-04-25-00:00:00::1, 2016-04-26-00:00:00::202, 2016-04-27-00:00:00::58, 2016-04-28-00:00:00::64, 2016-04-29-00:00:00::54, 2016-04-30-00:00:00::139, 2016-05-01-00:00:00::2, 2016-05-02-00:00:00::4, 2016-05-03-00:00:00::46, 2016-05-04-00:00:00::108, 2016-05-05-00:00:00::34, 2016-05-06-00:00:00::24, 2016-05-07-00:00:00::7, 2016-05-08-00:00:00::3, 2016-05-09-00:00:00::44, 2016-05-10-00:00:00::28, 2016-05-11-00:00:00::44, 2016-05-12-00:00:00::58, 2016-05-13-00:00:00::40, 2016-05-14-00:00:00::2, 2016-05-15-00:00:00::1, 2016-05-16-00:00:00::29, 2016-05-17-00:00:00::16, 2016-05-18-00:00:00::31, 2016-05-19-00:00:00::58, 2016-05-20-00:
 00:00::59, 2016-05-21-00:00:00::14, 2016-05-22-00:00:00::16, 2016-05-23-00:00:00::68, 2016-05-24-00:00:00::19, 2016-05-25-00:00:00::55, 2016-05-26-00:00:00::78, 2016-05-27-00:00:00::64, 2016-05-28-00:00:00::76, 2016-05-29-00:00:00::2, 2016-05-30-00:00:00::15, 2016-05-31-00:00:00::24, 2016-06-01-00:00:00::11, 2016-06-02-00:00:00::31, 2016-06-03-00:00:00::39, 2016-06-04-00:00:00::2, 2016-06-06-00:00:00::783, 2016-06-07-00:00:00::14, 2016-06-08-00:00:00::51, 2016-06-09-00:00:00::25, 2016-06-10-00:00:00::14, 2016-06-12-00:00:00::2, 2016-06-13-00:00:00::57, 2016-06-14-00:00:00::20, 2016-06-15-00:00:00::36, 2016-06-16-00:00:00::15, 2016-06-17-00:00:00::49, 2016-06-18-00:00:00::3, 2016-06-20-00:00:00::38, 2016-06-21-00:00:00::45, 2016-06-22-00:00:00::5215, 2016-06-23-00:00:00::4977, 2016-06-24-00:00:00::50, 2016-06-25-00:00:00::2, 2016-06-26-00:00:00::2, 2016-06-27-00:00:00::27, 2016-06-28-00:00:00::1102, 2016-06-29-00:00:00::59, 2016-06-30-00:00:00::38, 2016-07-01-00:00:00::34, 2016-07-02
 -00:00:00::1, 2016-07-03-00:00:00::3, 2016-07-04-00:00:00::69, 2016-07-05-00:00:00::26, 2016-07-06-00:00:00::5, 2016-07-07-00:00:00::5, 2016-07-08-00:00:00::26, 2016-07-09-00:00:00::58, 2016-07-10-00:00:00::3, 2016-07-11-00:00:00::13, 2016-07-12-00:00:00::1, 2016-07-13-00:00:00::68, 2016-07-14-00:00:00::73, 2016-07-15-00:00:00::69, 2016-07-16-00:00:00::65, 2016-07-17-00:00:00::4, 2016-07-18-00:00:00::1, 2016-07-19-00:00:00::14, 2016-07-20-00:00:00::60, 2016-07-21-00:00:00::328, 2016-07-22-00:00:00::41, 2016-07-23-00:00:00::105, 2016-07-24-00:00:00::16, 2016-07-25-00:00:00::1, 2016-07-26-00:00:00::37, 2016-07-27-00:00:00::72, 2016-07-28-00:00:00::59, 2016-07-29-00:00:00::53, 2016-07-30-00:00:00::241, 2016-07-31-00:00:00::1, 2016-08-01-00:00:00::10, 2016-08-02-00:00:00::2, 2016-08-03-00:00:00::45, 2016-08-04-00:00:00::44, 2016-08-05-00:00:00::76, 2016-08-06-00:00:00::96, 2016-08-07-00:00:00::8, 2016-08-08-00:00:00::129, 2016-08-09-00:00:00::14, 2016-08-10-00:00:00::50, 2016-08-11-00:0
 0:00::47, 2016-08-12-00:00:00::56, 2016-08-13-00:00:00::95, 2016-08-14-00:00:00::6, 2016-08-15-00:00:00::20, 2016-08-16-00:00:00::11, 2016-08-17-00:00:00::5, 2016-08-18-00:00:00::94, 2016-08-19-00:00:00::51, 2016-08-20-00:00:00::110, 2016-08-21-00:00:00::2, 2016-08-22-00:00:00::30, 2016-08-23-00:00:00::3, 2016-08-24-00:00:00::50, 2016-08-25-00:00:00::55, 2016-08-26-00:00:00::51, 2016-08-27-00:00:00::103, 2016-08-28-00:00:00::5, 2016-08-29-00:00:00::6, 2016-08-31-00:00:00::58, 2016-09-01-00:00:00::72, 2016-09-02-00:00:00::94, 2016-09-03-00:00:00::107, 2016-09-04-00:00:00::9, 2016-09-05-00:00:00::1, 2016-09-06-00:00:00::10, 2016-09-07-00:00:00::58, 2016-09-08-00:00:00::82, 2016-09-09-00:00:00::44, 2016-09-10-00:00:00::162, 2016-09-11-00:00:00::5, 2016-09-12-00:00:00::5, 2016-09-13-00:00:00::1, 2016-09-14-00:00:00::64, 2016-09-15-00:00:00::62, 2016-09-16-00:00:00::55, 2016-09-17-00:00:00::106, 2016-09-18-00:00:00::17, 2016-09-19-00:00:00::6, 2016-09-20-00:00:00::4, 2016-09-21-00:00:00:
 :68, 2016-09-22-00:00:00::69, 2016-09-23-00:00:00::62, 2016-09-24-00:00:00::101, 2016-09-25-00:00:00::11, 2016-09-26-00:00:00::13, 2016-09-27-00:00:00::2, 2016-09-28-00:00:00::94, 2016-09-29-00:00:00::42, 2016-09-30-00:00:00::47, 2016-10-01-00:00:00::109, 2016-10-02-00:00:00::13, 2016-10-03-00:00:00::2, 2016-10-04-00:00:00::43, 2016-10-05-00:00:00::38, 2016-10-06-00:00:00::41, 2016-10-07-00:00:00::31, 2016-10-08-00:00:00::64, 2016-10-09-00:00:00::7, 2016-10-10-00:00:00::13, 2016-10-11-00:00:00::2, 2016-10-12-00:00:00::47, 2016-10-13-00:00:00::49, 2016-10-14-00:00:00::114, 2016-10-15-00:00:00::96, 2016-10-16-00:00:00::9, 2016-10-17-00:00:00::3, 2016-10-18-00:00:00::3, 2016-10-19-00:00:00::60, 2016-10-20-00:00:00::9, 2016-10-21-00:00:00::160, 2016-10-22-00:00:00::53, 2016-10-23-00:00:00::2, 2016-10-24-00:00:00::4, 2016-10-25-00:00:00::4, 2016-10-26-00:00:00::69, 2016-10-27-00:00:00::67, 2016-10-28-00:00:00::4, 2016-10-29-00:00:00::182, 2016-10-31-00:00:00::8, 2016-11-01-00:00:00::72, 
 2016-11-02-00:00:00::68, 2016-11-03-00:00:00::53, 2016-11-04-00:00:00::54, 2016-11-05-00:00:00::94, 2016-11-06-00:00:00::2, 2016-11-07-00:00:00::24, 2016-11-08-00:00:00::14, 2016-11-09-00:00:00::83, 2016-11-10-00:00:00::71, 2016-11-11-00:00:00::74, 2016-11-12-00:00:00::129, 2016-11-13-00:00:00::3, 2016-11-14-00:00:00::4, 2016-11-16-00:00:00::55, 2016-11-17-00:00:00::54, 2016-11-18-00:00:00::77, 2016-11-19-00:00:00::119, 2016-11-20-00:00:00::4, 2016-11-21-00:00:00::14, 2016-11-22-00:00:00::3, 2016-11-23-00:00:00::52, 2016-11-24-00:00:00::33, 2016-11-25-00:00:00::45, 2016-11-26-00:00:00::66, 2016-11-27-00:00:00::4, 2016-11-28-00:00:00::3, 2016-11-29-00:00:00::1, 2016-11-30-00:00:00::60, 2016-12-01-00:00:00::78, 2016-12-02-00:00:00::84, 2016-12-03-00:00:00::66, 2016-12-04-00:00:00::7, 2016-12-05-00:00:00::38, 2016-12-06-00:00:00::53, 2016-12-07-00:00:00::147, 2016-12-08-00:00:00::472, 2016-12-09-00:00:00::112, 2016-12-10-00:00:00::80, 2016-12-11-00:00:00::8, 2016-12-12-00:00:00::9, 201
 6-12-14-00:00:00::42, 2016-12-16-00:00:00::76, 2016-12-17-00:00:00::105, 2016-12-18-00:00:00::5, 2016-12-19-00:00:00::8, 2016-12-21-00:00:00::63, 2016-12-22-00:00:00::31, 2016-12-23-00:00:00::56, 2016-12-24-00:00:00::51, 2016-12-25-00:00:00::9, 2016-12-26-00:00:00::1, 2016-12-28-00:00:00::3, 2016-12-30-00:00:00::7, 2016-12-31-00:00:00::4, 2017-01-02-00:00:00::1, 2017-01-04-00:00:00::10, 2017-01-05-00:00:00::57, 2017-01-06-00:00:00::133, 2017-01-07-00:00:00::99, 2017-01-08-00:00:00::5, 2017-01-09-00:00:00::5, 2017-01-10-00:00:00::14, 2017-01-11-00:00:00::61, 2017-01-12-00:00:00::39, 2017-01-13-00:00:00::55, 2017-01-14-00:00:00::107, 2017-01-15-00:00:00::5, 2017-01-16-00:00:00::3, 2017-01-18-00:00:00::39, 2017-01-19-00:00:00::48, 2017-01-20-00:00:00::38, 2017-01-21-00:00:00::73, 2017-01-22-00:00:00::6, 2017-01-23-00:00:00::10, 2017-01-25-00:00:00::1, 2017-01-26-00:00:00::104, 2017-01-27-00:00:00::1, 2017-01-28-00:00:00::117, 2017-01-29-00:00:00::3, 2017-01-30-00:00:00::4, 2017-01-31-0
 0:00:00::78, 2017-02-01-00:00:00::60, 2017-02-02-00:00:00::49, 2017-02-03-00:00:00::46, 2017-02-04-00:00:00::113, 2017-02-05-00:00:00::6, 2017-02-06-00:00:00::56, 2017-02-07-00:00:00::302, 2017-02-08-00:00:00::75, 2017-02-09-00:00:00::54, 2017-02-10-00:00:00::226, 2017-02-11-00:00:00::1039, 2017-02-13-00:00:00::29515, 2017-02-14-00:00:00::602, 2017-02-15-00:00:00::1099, 2017-02-16-00:00:00::97, 2017-02-17-00:00:00::1608, 2017-02-18-00:00:00::1258, 2017-02-19-00:00:00::109, 2017-02-20-00:00:00::116, 2017-02-21-00:00:00::336, 2017-02-22-00:00:00::664, 2017-02-23-00:00:00::154529, 2017-02-24-00:00:00::40521, 2017-02-25-00:00:00::2100, 2017-02-26-00:00:00::131, 2017-02-27-00:00:00::415645, 2017-02-28-00:00:00::276943, 2017-03-01-00:00:00::3306, 2017-03-02-00:00:00::3552, 2017-03-03-00:00:00::4227, 2017-03-04-00:00:00::6963, 2017-03-05-00:00:00::440, 2017-03-06-00:00:00::3365, 2017-03-07-00:00:00::389, 2017-03-08-00:00:00::3870, 2017-03-09-00:00:00::3867, 2017-03-10-00:00:00::4218, 2017-
 03-11-00:00:00::7977, 2017-03-12-00:00:00::391, 2017-03-13-00:00:00::27355, 2017-03-14-00:00:00::34284, 2017-03-15-00:00:00::14698, 2017-03-16-00:00:00::2926, 2017-03-17-00:00:00::5338, 2017-03-18-00:00:00::7752, 2017-03-19-00:00:00::510, 2017-03-20-00:00:00::7459, 2017-03-21-00:00:00::440, 2017-03-22-00:00:00::6040, 2017-03-23-00:00:00::6313, 2017-03-24-00:00:00::5792, 2017-03-25-00:00:00::9397, 2017-03-26-00:00:00::613, 2017-03-27-00:00:00::1365, 2017-03-28-00:00:00::7063, 2017-03-29-00:00:00::2110, 2017-03-30-00:00:00::8154, 2017-03-31-00:00:00::9777, 2017-04-01-00:00:00::16354, 2017-04-02-00:00:00::1288, 2017-04-03-00:00:00::1426, 2017-04-04-00:00:00::2666, 2017-04-05-00:00:00::13562, 2017-04-06-00:00:00::54698, 2017-04-07-00:00:00::22457";
 }