Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

GOBBLIN-573: Add option to use finer level granularity at the hour level for TimeAwareDatasetfinder #2438

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,14 +33,14 @@
public class TimeAwareRecursiveCopyableDataset extends RecursiveCopyableDataset {
private static final String CONFIG_PREFIX = CopyConfiguration.COPY_PREFIX + ".recursive";
public static final String DATE_PATTERN_KEY = CONFIG_PREFIX + ".date.pattern";
public static final String LOOKBACK_DAYS_KEY = CONFIG_PREFIX + ".lookback.days";
public static final String LOOKBACK_TIME_KEY = CONFIG_PREFIX + ".lookback.time";

private final Integer lookbackDays;
private final String lookbackTime;
private final String datePattern;

public TimeAwareRecursiveCopyableDataset(FileSystem fs, Path rootPath, Properties properties, Path glob) {
super(fs, rootPath, properties, glob);
this.lookbackDays = Integer.parseInt(properties.getProperty(LOOKBACK_DAYS_KEY));
this.lookbackTime = properties.getProperty(LOOKBACK_TIME_KEY);
this.datePattern = properties.getProperty(DATE_PATTERN_KEY);
}

Expand Down Expand Up @@ -86,7 +86,14 @@ public void remove() {
protected List<FileStatus> getFilesAtPath(FileSystem fs, Path path, PathFilter fileFilter) throws IOException {
DateTimeFormatter formatter = DateTimeFormatter.ofPattern(datePattern);
LocalDateTime endDate = LocalDateTime.now();
LocalDateTime startDate = endDate.minusDays(lookbackDays);
LocalDateTime startDate;
if (this.lookbackTime.endsWith("d")) {
startDate = endDate.minusDays(Long.parseLong(lookbackTime.substring(0, lookbackTime.length() - 1)));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What about using Period like in SelectBetweenTimeBasedPolicy? I think that allows mixing like 7d6h.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good suggestion! Thanks!

} else if (this.lookbackTime.endsWith("h")) {
startDate = endDate.minusHours(Long.parseLong(lookbackTime.substring(0, lookbackTime.length() - 1)));
} else {
startDate = endDate.minusDays(Long.parseLong(lookbackTime));
}
DateRangeIterator dateRangeIterator = new DateRangeIterator(startDate, endDate, datePattern);
List<FileStatus> fileStatuses = Lists.newArrayList();
while (dateRangeIterator.hasNext()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* 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.time.LocalDateTime;
import java.time.format.DateTimeFormatter;
import java.util.List;
import java.util.Properties;

import org.apache.hadoop.conf.Configuration;
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.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

import lombok.extern.slf4j.Slf4j;

import org.apache.gobblin.util.filters.HiddenFilter;

@Slf4j
public class TimeAwareRecursiveCopyableDatasetTest {
private FileSystem fs;
private Path baseDir1;
private Path baseDir2;

@BeforeClass
public void setUp() throws IOException {
this.fs = FileSystem.getLocal(new Configuration());

baseDir1 = new Path("/tmp/src/ds1/hourly");
if (fs.exists(baseDir1)) {
fs.delete(baseDir1, true);
}
fs.mkdirs(baseDir1);

baseDir2 = new Path("/tmp/src/ds1/daily");
if (fs.exists(baseDir2)) {
fs.delete(baseDir2, true);
}
fs.mkdirs(baseDir2);
}

@Test
public void testGetFilesAtPath() throws IOException {
String datePattern = "yyyy/MM/dd/HH";
DateTimeFormatter formatter = DateTimeFormatter.ofPattern(datePattern);

LocalDateTime endDate = LocalDateTime.now();

for (int i = 0; i < 24; i++) {
String startDate = endDate.minusHours(i).format(formatter);
Path subDirPath = new Path(baseDir1, new Path(startDate));
fs.mkdirs(subDirPath);
fs.create(new Path(subDirPath, i + ".avro"));
}

Properties properties = new Properties();
properties.setProperty(TimeAwareRecursiveCopyableDataset.LOOKBACK_TIME_KEY, "4h");
properties.setProperty(TimeAwareRecursiveCopyableDataset.DATE_PATTERN_KEY, "yyyy/MM/dd/HH");

PathFilter pathFilter = new HiddenFilter();
TimeAwareRecursiveCopyableDataset dataset = new TimeAwareRecursiveCopyableDataset(fs, baseDir1, properties,
new Path("/tmp/src/*/hourly"));
List<FileStatus> fileStatusList = dataset.getFilesAtPath(fs, baseDir1, pathFilter);

Assert.assertEquals(fileStatusList.size(), 5);

datePattern = "yyyy/MM/dd";
formatter = DateTimeFormatter.ofPattern(datePattern);
endDate = LocalDateTime.now();

for (int i = 0; i < 3; i++) {
String startDate = endDate.minusDays(i).format(formatter);
Path subDirPath = new Path(baseDir2, new Path(startDate));
fs.mkdirs(subDirPath);
fs.create(new Path(subDirPath, i + ".avro"));
}

properties = new Properties();
properties.setProperty(TimeAwareRecursiveCopyableDataset.LOOKBACK_TIME_KEY, "2d");
properties.setProperty(TimeAwareRecursiveCopyableDataset.DATE_PATTERN_KEY, "yyyy/MM/dd");

dataset = new TimeAwareRecursiveCopyableDataset(fs, baseDir2, properties,
new Path("/tmp/src/*/daily"));
fileStatusList = dataset.getFilesAtPath(fs, baseDir2, pathFilter);

Assert.assertEquals(fileStatusList.size(), 3);
}

@AfterClass
public void clean() throws IOException {
//Delete tmp directories
this.fs.delete(baseDir1, true);
this.fs.delete(baseDir2, true);
}
}