[ 
https://issues.apache.org/jira/browse/GOBBLIN-910?focusedWorklogId=328877&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-328877
 ]

ASF GitHub Bot logged work on GOBBLIN-910:
------------------------------------------

                Author: ASF GitHub Bot
            Created on: 16/Oct/19 00:28
            Start Date: 16/Oct/19 00:28
    Worklog Time Spent: 10m 
      Work Description: sv2000 commented on pull request #2765: [GOBBLIN-910] 
Added a unix timestamp recursive copyable dataset finder
URL: https://github.com/apache/incubator-gobblin/pull/2765#discussion_r335232277
 
 

 ##########
 File path: 
gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/UnixTimestampRecursiveCopyableDataset.java
 ##########
 @@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.data.management.copy;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.joda.time.DateTimeZone;
+import org.joda.time.LocalDate;
+import org.joda.time.LocalDateTime;
+import org.joda.time.Period;
+import org.joda.time.format.PeriodFormatter;
+import org.joda.time.format.PeriodFormatterBuilder;
+import org.testng.collections.Lists;
+
+import org.apache.gobblin.util.filters.AndPathFilter;
+
+
+/**
+ * This dataset reads files with format /path/to/dir/".*([0-9]{13})-PT-.*"*
+ * where the regex can be supplied through configuration.
+ * It extracts the timestamp from the path and filters paths which lie within 
the {@link #lookbackPeriod}
+ * It then applies the {@link #snapshotSelectionPolicy} to filter further.
+ */
+public class UnixTimestampRecursiveCopyableDataset extends 
RecursiveCopyableDataset {
+
+  private static final String CONFIG_PREFIX = CopyConfiguration.COPY_PREFIX + 
".recursive";
+  public static final String SNAPSHOT_SELECTION_POLICY = CONFIG_PREFIX + 
".snapshot.selection.policy";
+  public static final String TIMESTAMP_REGEEX = CONFIG_PREFIX + 
".timestamp.pattern";
+  public static final String DEFAULT_TIMESTAMP_REGEEX = ".*([0-9]{13})-PT-.*";
+  private final String lookbackTime;
+  private final Period lookbackPeriod;
+  private final LocalDateTime currentTime;
+  private final SnapshotSelectionPolicy snapshotSelectionPolicy;
+  private final DateTimeZone dateTimeZone;
+  private final Pattern timestampPattern;
+
+  public UnixTimestampRecursiveCopyableDataset(FileSystem fs, Path rootPath, 
Properties properties, Path glob) {
+    super(fs, rootPath, properties, glob);
+    this.lookbackTime = 
properties.getProperty(TimeAwareRecursiveCopyableDataset.LOOKBACK_TIME_KEY);
+    this.snapshotSelectionPolicy =
+        
SnapshotSelectionPolicy.valueOf(properties.getProperty(SNAPSHOT_SELECTION_POLICY).toUpperCase());
+    PeriodFormatter periodFormatter = new 
PeriodFormatterBuilder().appendDays().appendSuffix("d").toFormatter();
+    this.lookbackPeriod = periodFormatter.parsePeriod(lookbackTime);
+    String timestampRegex = properties.getProperty(TIMESTAMP_REGEEX, 
DEFAULT_TIMESTAMP_REGEEX);
+    this.timestampPattern = Pattern.compile(timestampRegex);
+    this.dateTimeZone = DateTimeZone.forID(properties
+        
.getProperty(TimeAwareRecursiveCopyableDataset.DATE_PATTERN_TIMEZONE_KEY,
+            TimeAwareRecursiveCopyableDataset.DEFAULT_DATE_PATTERN_TIMEZONE));
+    this.currentTime = LocalDateTime.now(this.dateTimeZone);
+  }
+
+  private enum SnapshotSelectionPolicy {
+    EARLIEST, LATEST, ALL
+  }
+
+  class TimestampPathFilter implements PathFilter {
+
+    @Override
+    public boolean accept(Path path) {
+
+      LocalDate endDate = currentTime.toLocalDate();
+      LocalDate startDate = endDate.minus(lookbackPeriod);
+      Matcher matcher = timestampPattern.matcher(path.toString());
+      if (!matcher.matches()) {
+        return false;
+      }
+      Long timestamp = Long.parseLong(matcher.group(1));
+      LocalDate dateOfTimestamp = new LocalDateTime(timestamp, 
dateTimeZone).toLocalDate();
+      return !(dateOfTimestamp == null || dateOfTimestamp.isAfter(endDate) || 
dateOfTimestamp.isEqual(startDate)
+          || dateOfTimestamp.isBefore(startDate));
+    }
+  }
+
+  @Override
+  protected List<FileStatus> getFilesAtPath(FileSystem fs, Path path, 
PathFilter fileFilter)
+      throws IOException {
+
+    // Filter files by lookback period (fileNames >= startDate and fileNames 
<= endDate)
+    PathFilter andPathFilter = new AndPathFilter(fileFilter, new 
TimestampPathFilter());
+    List<FileStatus> files = super.getFilesAtPath(fs, path, andPathFilter);
+
+    if (SnapshotSelectionPolicy.ALL == snapshotSelectionPolicy) {
+      return files;
+    }
+
+    Map<String, Map<LocalDate, Long>> pathDateTimestampMap = new HashMap<>();
+
+    // Now select files per day based on snapshot selection policy
+    for (FileStatus fileStatus : files) {
+      String fullPath = fileStatus.getPath().toString();
+      Matcher matcher = timestampPattern.matcher(fullPath);
+      if (!matcher.matches()) {
+        continue;
+      }
+      String timestampStr = matcher.group(1);
+      String pathToSnapshotFolder = fullPath.substring(0, 
fullPath.indexOf(timestampStr));
+      if (!pathDateTimestampMap.containsKey(pathToSnapshotFolder)) {
+        pathDateTimestampMap.put(pathToSnapshotFolder, new HashMap<>());
+      }
+      Map<LocalDate, Long> dateFileStatusMap = 
pathDateTimestampMap.get(pathToSnapshotFolder);
+      Long unixTimestamp = Long.parseLong(timestampStr);
+      LocalDate fileDate = new LocalDateTime(unixTimestamp).toLocalDate();
+      if (!dateFileStatusMap.containsKey(fileDate)) {
+        dateFileStatusMap.put(fileDate, unixTimestamp);
+        continue;
+      }
+      Long fromMap = dateFileStatusMap.get(fileDate);
+
+      switch (snapshotSelectionPolicy) {
 
 Review comment:
   Will we throw an exception when snapshot selection policy is ALL?
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 328877)
    Time Spent: 50m  (was: 40m)

> Add Unix timestamp copyable recursive dataset finder with snapshot selection 
> policy
> -----------------------------------------------------------------------------------
>
>                 Key: GOBBLIN-910
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-910
>             Project: Apache Gobblin
>          Issue Type: New Feature
>            Reporter: Vikram Bohra
>            Priority: Major
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> A dataset finder that takes a lookback time and a snapshot selection p[olicy 
> to return a list of files to copy from source to destination. 
> The finder also takes in a regex that is used to extract the timestamp from 
> the path.
> snapshot policies 
> ALL: Returns all files within the look back time 
> EARLIEST: returns the earliest snapshot per day of the look back time
> LATEST: returns the last snapshot per day of the look back time



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to