Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@

import java.io.IOException;
import java.io.Serializable;
import java.time.Instant;
import java.time.ZonedDateTime;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -523,8 +522,7 @@ public Option<HoodieInstant> getEarliestCommitToRetain() {
- commitsRetained); //15 instants total, 10 commits to retain, this gives 6th instant in the list
}
} else if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) {
Instant instant = Instant.now();
ZonedDateTime currentDateTime = ZonedDateTime.ofInstant(instant, hoodieTable.getMetaClient().getTableConfig().getTimelineTimezone().getZoneId());
ZonedDateTime currentDateTime = ZonedDateTime.now();
String earliestTimeToRetain = HoodieActiveTimeline.formatDate(Date.from(currentDateTime.minusHours(hoursRetained).toInstant()));
earliestCommitToRetain = Option.fromJavaOptional(commitTimeline.getInstantsAsStream().filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(),
HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestTimeToRetain)).findFirst());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.common.table.timeline;

import org.apache.hudi.common.model.HoodieTimelineTimeZone;
import org.apache.hudi.common.table.HoodieTableConfig;

import java.text.ParseException;
import java.time.LocalDateTime;
Expand All @@ -44,10 +45,10 @@ public class HoodieInstantTimeGenerator {
public static final int MILLIS_INSTANT_TIMESTAMP_FORMAT_LENGTH = MILLIS_INSTANT_TIMESTAMP_FORMAT.length();
// Formatter to generate Instant timestamps
// Unfortunately millisecond format is not parsable as is https://bugs.openjdk.java.net/browse/JDK-8031085. hence have to do appendValue()
private static DateTimeFormatter MILLIS_INSTANT_TIME_FORMATTER = new DateTimeFormatterBuilder().appendPattern(SECS_INSTANT_TIMESTAMP_FORMAT)
private static final DateTimeFormatter MILLIS_INSTANT_TIME_FORMATTER = new DateTimeFormatterBuilder().appendPattern(SECS_INSTANT_TIMESTAMP_FORMAT)
.appendValue(ChronoField.MILLI_OF_SECOND, 3).toFormatter();
private static final String MILLIS_GRANULARITY_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS";
private static DateTimeFormatter MILLIS_GRANULARITY_DATE_FORMATTER = DateTimeFormatter.ofPattern(MILLIS_GRANULARITY_DATE_FORMAT);
private static final DateTimeFormatter MILLIS_GRANULARITY_DATE_FORMATTER = DateTimeFormatter.ofPattern(MILLIS_GRANULARITY_DATE_FORMAT);

// The last Instant timestamp generated
private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE));
Expand All @@ -61,7 +62,7 @@ public class HoodieInstantTimeGenerator {

/**
* Returns next instant time that adds N milliseconds to the current time.
* Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
* Ensures each instant time is at least 1 second apart since we create instant times at second granularity
*
* @param milliseconds Milliseconds to add to current time while generating the new instant time
*/
Expand Down Expand Up @@ -94,7 +95,7 @@ public static Date parseDateFromInstantTime(String timestamp) throws ParseExcept
}

LocalDateTime dt = LocalDateTime.parse(timestampInMillis, MILLIS_INSTANT_TIME_FORMATTER);
return Date.from(dt.atZone(ZoneId.systemDefault()).toInstant());
return Date.from(dt.atZone(getZoneId()).toInstant());
} catch (DateTimeParseException e) {
throw new ParseException(e.getMessage(), e.getErrorIndex());
}
Expand Down Expand Up @@ -129,7 +130,7 @@ public static String getInstantForDateString(String dateString) {
}

private static TemporalAccessor convertDateToTemporalAccessor(Date d) {
return d.toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime();
return d.toInstant().atZone(getZoneId()).toLocalDateTime();
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Can we supplement some UTs for parseDateFromInstantTime and convertDateToTemporalAccessor ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can we supplement some UTs for parseDateFromInstantTime and convertDateToTemporalAccessor ?

I would be happy to do it.

Copy link
Contributor Author

@xiaochen-zhou xiaochen-zhou May 9, 2023

Choose a reason for hiding this comment

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

convertDateToTemporalAccessor

I added two UTs: testFormatDateWithCommitTimeZone and testInstantDateParsingWithCommitTimeZone, testInstantDateParsingWithCommitTimeZone is used to test the correctness of the HoodieInstantTimeGenerator#convertDateToTemporalAccessor()

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can we supplement some UTs for parseDateFromInstantTime and convertDateToTemporalAccessor ?

And in the TestHoodieActiveTimeline.java, there are many UTs related to DateParsing, such as:

  • testInvalidInstantDateParsing
  • testMillisGranularityInstantDateParsing
    etc.

public static void setCommitTimeZone(HoodieTimelineTimeZone commitTimeZone) {
Expand All @@ -144,4 +145,11 @@ public static boolean isValidInstantTime(String instantTime) {
return false;
}
}

private static ZoneId getZoneId() {
HoodieTimelineTimeZone timelineTimezone = new HoodieTableConfig().getTimelineTimezone();
return timelineTimezone.equals(HoodieTimelineTimeZone.LOCAL)
? ZoneId.systemDefault()
Copy link
Contributor

@danny0405 danny0405 May 10, 2023

Choose a reason for hiding this comment

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

If possible, fetch the timezone with metaClient.tableConfig, the HoodieTimelineTimeZone can not assure the initialization of zoneId.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If possible, fetch the timezone whout metaClient.tableConfig, the HoodieTimelineTimeZone can not assure the initialization of zoneId.

I will try to modify the code as you say

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If possible, fetch the timezone whout metaClient.tableConfig, the HoodieTimelineTimeZone can not assure the initialization of zoneId.

In the class HoodieInstantTimeGenerator, set an initial value( HoodieTimelineTimeZone.LOCAL ) for the property commitTimeZone

private static HoodieTimelineTimeZone commitTimeZone = HoodieTimelineTimeZone.LOCAL;

And update commitTimeZone value in HoodieTableConfig#create

if (hoodieConfig.contains(TIMELINE_TIMEZONE)) {
    HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getString(TIMELINE_TIMEZONE)));
}
public static void setCommitTimeZone(HoodieTimelineTimeZone commitTimeZone) {
  HoodieInstantTimeGenerator.commitTimeZone = commitTimeZone;
}

So, I think getting ZoneId by HoodieTimelineTimeZone should be correct. and I don't really understand the meaning of the HoodieTimelineTimeZone can not assure the initialization of zoneId.
I don't know if my idea is correct, looking forward to your reply.

Copy link
Contributor

Choose a reason for hiding this comment

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

See the discussions we take in: #8631

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See the discussions we take in: #8631

I sees, I will try to modify the code as you say.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See the discussions we take in: #8631

It seems that there is no good way to get HoodieTimelineTimeZone through HoodieTableMetaClient in HoodieInstantTimeGenerator, I currently get HoodieTimelineTimeZone by instantiate a HoodieTableConfig, can you give me some advice?

Copy link
Contributor

@danny0405 danny0405 May 15, 2023

Choose a reason for hiding this comment

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

I currently get HoodieTimelineTimeZone by instantiate a HoodieTableConfig

If no existing table meta client or table config can be reused, we must instantiate a new one. For HoodieTableConfig, usually we fetch a meta client first then get the config, take

public static Option<HoodieTableConfig> getTableConfig(String basePath, org.apache.hadoop.conf.Configuration hadoopConf) {
for a reference.

: ZoneId.of(commitTimeZone.getTimeZone().toUpperCase());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.fs.NoOpConsistencyGuard;
import org.apache.hudi.common.model.HoodieTimelineTimeZone;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
Expand All @@ -37,6 +38,8 @@
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
Expand Down Expand Up @@ -648,6 +651,22 @@ public void testInvalidInstantDateParsing() throws ParseException {
HoodieActiveTimeline.parseDateFromInstantTimeSafely(testInstant).get().getTime());
}

@Test
public void testFormatDateWithCommitTimeZone() throws ParseException {
String testInstant = "20230508120101";
Date date = HoodieActiveTimeline.parseDateFromInstantTime(testInstant);

HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.LOCAL);
LocalDateTime localDateTime = date.toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime();
String defaultStr = HoodieInstantTimeGenerator.getInstantFromTemporalAccessor(localDateTime);
assertEquals(defaultStr, HoodieActiveTimeline.formatDate(date));

HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.UTC);
LocalDateTime utcDateTime = date.toInstant().atZone(ZoneId.of("UTC")).toLocalDateTime();
String utcStr = HoodieInstantTimeGenerator.getInstantFromTemporalAccessor(utcDateTime);
assertEquals(utcStr, HoodieActiveTimeline.formatDate(date));
}

/**
* Returns an exhaustive list of all possible HoodieInstant.
* @return list of HoodieInstant
Expand Down