Skip to content

Commit

Permalink
[HUDI-1587] Add latency and freshness support (#2541)
Browse files Browse the repository at this point in the history
Save min and max of event time in each commit and compute the latency and freshness metrics.
  • Loading branch information
xushiyan committed Mar 4, 2021
1 parent f11a6c7 commit 899ae70
Show file tree
Hide file tree
Showing 14 changed files with 281 additions and 24 deletions.
Expand Up @@ -21,20 +21,28 @@
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.util.DateTimeUtils;
import org.apache.hudi.common.util.Option;

import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import java.io.Serializable;
import java.time.DateTimeException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;

import static org.apache.hudi.common.model.DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY;

/**
* Status of a write operation.
*/
public class WriteStatus implements Serializable {

private static final Logger LOG = LogManager.getLogger(WriteStatus.class);
private static final long serialVersionUID = 1L;
private static final long RANDOM_SEED = 9038412832L;

Expand Down Expand Up @@ -77,6 +85,18 @@ public void markSuccess(HoodieRecord record, Option<Map<String, String>> optiona
writtenRecords.add(record);
}
totalRecords++;

// get the min and max event time for calculating latency and freshness
if (optionalRecordMetadata.isPresent()) {
String eventTimeVal = optionalRecordMetadata.get().getOrDefault(METADATA_EVENT_TIME_KEY, null);
try {
long eventTime = DateTimeUtils.parseDateTime(eventTimeVal).toEpochMilli();
stat.setMinEventTime(eventTime);
stat.setMaxEventTime(eventTime);
} catch (DateTimeException | IllegalArgumentException e) {
LOG.debug(String.format("Fail to parse event time value: %s", eventTimeVal), e);
}
}
}

/**
Expand Down
Expand Up @@ -19,14 +19,15 @@
package org.apache.hudi.config;

import org.apache.hudi.common.config.DefaultHoodieConfig;
import org.apache.hudi.config.HoodieMemoryConfig.Builder;

import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;

import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL;
import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_ORDERING_FIELD_VAL;
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP;
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP;

/**
Expand Down Expand Up @@ -63,10 +64,17 @@ public Builder withPayloadOrderingField(String payloadOrderingField) {
return this;
}

public Builder withPayloadEventTimeField(String payloadEventTimeField) {
props.setProperty(PAYLOAD_EVENT_TIME_FIELD_PROP, String.valueOf(payloadEventTimeField));
return this;
}

public HoodiePayloadConfig build() {
HoodiePayloadConfig config = new HoodiePayloadConfig(props);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_ORDERING_FIELD_PROP), PAYLOAD_ORDERING_FIELD_PROP,
String.valueOf(DEFAULT_PAYLOAD_ORDERING_FIELD_VAL));
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_EVENT_TIME_FIELD_PROP), PAYLOAD_EVENT_TIME_FIELD_PROP,
String.valueOf(DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL));
return config;
}
}
Expand Down
Expand Up @@ -20,6 +20,8 @@

import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;

import com.codahale.metrics.Timer;
Expand Down Expand Up @@ -130,6 +132,7 @@ public Timer.Context getIndexCtx() {

public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata,
String actionType) {
updateCommitTimingMetrics(commitEpochTimeInMs, durationInMs, metadata, actionType);
if (config.isMetricsOn()) {
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
long totalFilesInsert = metadata.fetchTotalFilesInsert();
Expand All @@ -144,15 +147,13 @@ public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, Hoo
long totalCompactedRecordsUpdated = metadata.getTotalCompactedRecordsUpdated();
long totalLogFilesCompacted = metadata.getTotalLogFilesCompacted();
long totalLogFilesSize = metadata.getTotalLogFilesSize();
Metrics.registerGauge(getMetricsName(actionType, "duration"), durationInMs);
Metrics.registerGauge(getMetricsName(actionType, "totalPartitionsWritten"), totalPartitionsWritten);
Metrics.registerGauge(getMetricsName(actionType, "totalFilesInsert"), totalFilesInsert);
Metrics.registerGauge(getMetricsName(actionType, "totalFilesUpdate"), totalFilesUpdate);
Metrics.registerGauge(getMetricsName(actionType, "totalRecordsWritten"), totalRecordsWritten);
Metrics.registerGauge(getMetricsName(actionType, "totalUpdateRecordsWritten"), totalUpdateRecordsWritten);
Metrics.registerGauge(getMetricsName(actionType, "totalInsertRecordsWritten"), totalInsertRecordsWritten);
Metrics.registerGauge(getMetricsName(actionType, "totalBytesWritten"), totalBytesWritten);
Metrics.registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs);
Metrics.registerGauge(getMetricsName(actionType, "totalScanTime"), totalTimeTakenByScanner);
Metrics.registerGauge(getMetricsName(actionType, "totalCreateTime"), totalTimeTakenForInsert);
Metrics.registerGauge(getMetricsName(actionType, "totalUpsertTime"), totalTimeTakenForUpsert);
Expand All @@ -162,6 +163,23 @@ public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, Hoo
}
}

private void updateCommitTimingMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata,
String actionType) {
if (config.isMetricsOn()) {
Pair<Option<Long>, Option<Long>> eventTimePairMinMax = metadata.getMinAndMaxEventTime();
if (eventTimePairMinMax.getLeft().isPresent()) {
long commitLatencyInMs = commitEpochTimeInMs + durationInMs - eventTimePairMinMax.getLeft().get();
Metrics.registerGauge(getMetricsName(actionType, "commitLatencyInMs"), commitLatencyInMs);
}
if (eventTimePairMinMax.getRight().isPresent()) {
long commitFreshnessInMs = commitEpochTimeInMs + durationInMs - eventTimePairMinMax.getRight().get();
Metrics.registerGauge(getMetricsName(actionType, "commitFreshnessInMs"), commitFreshnessInMs);
}
Metrics.registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs);
Metrics.registerGauge(getMetricsName(actionType, "duration"), durationInMs);
}
}

public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
if (config.isMetricsOn()) {
LOG.info(
Expand Down
Expand Up @@ -19,6 +19,8 @@
package org.apache.hudi.metrics;

import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;

import com.codahale.metrics.Timer;
Expand Down Expand Up @@ -123,6 +125,7 @@ public void testTimerCtx() throws InterruptedException {
when(metadata.getTotalCompactedRecordsUpdated()).thenReturn(randomValue + 11);
when(metadata.getTotalLogFilesCompacted()).thenReturn(randomValue + 12);
when(metadata.getTotalLogFilesSize()).thenReturn(randomValue + 13);
when(metadata.getMinAndMaxEventTime()).thenReturn(Pair.of(Option.empty(), Option.empty()));
metrics.updateCommitMetrics(randomValue + 14, commitTimer.stop(), metadata, action);

String metricname = metrics.getMetricsName(action, "duration");
Expand Down
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.client;

import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;

import org.junit.jupiter.api.Test;

Expand All @@ -44,8 +45,8 @@ public void testSuccessRecordTracking() {
WriteStatus status = new WriteStatus(false, 1.0);
Throwable t = new Exception("some error in writing");
for (int i = 0; i < 1000; i++) {
status.markSuccess(mock(HoodieRecord.class), null);
status.markFailure(mock(HoodieRecord.class), t, null);
status.markSuccess(mock(HoodieRecord.class), Option.empty());
status.markFailure(mock(HoodieRecord.class), t, Option.empty());
}
assertEquals(1000, status.getFailedRecords().size());
assertTrue(status.hasErrors());
Expand Down
Expand Up @@ -25,6 +25,8 @@
import org.apache.avro.generic.IndexedRecord;

import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;

import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
Expand All @@ -37,6 +39,9 @@
*/
public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {

public static final String METADATA_EVENT_TIME_KEY = "metadata.event_time.key";
private Option<Object> eventTime = Option.empty();

public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {
super(record, orderingVal);
}
Expand Down Expand Up @@ -71,6 +76,10 @@ public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue

/*
* We reached a point where the value is disk is older than the incoming record.
*/
eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP), true));

/*
* Now check if the incoming record is a delete record.
*/
if (isDeleteRecord(incomingRecord)) {
Expand All @@ -79,4 +88,13 @@ public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue
return Option.of(incomingRecord);
}
}

@Override
public Option<Map<String, String>> getMetadata() {
Map<String, String> metadata = new HashMap<>();
if (eventTime.isPresent()) {
metadata.put(METADATA_EVENT_TIME_KEY, String.valueOf(eventTime.get()));
}
return metadata.isEmpty() ? Option.empty() : Option.of(metadata);
}
}
Expand Up @@ -18,14 +18,16 @@

package org.apache.hudi.common.model;

import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;

import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

Expand Down Expand Up @@ -323,6 +325,20 @@ public Long getTotalUpsertTime() {
return totalUpsertTime;
}

public Pair<Option<Long>, Option<Long>> getMinAndMaxEventTime() {
long minEventTime = Long.MAX_VALUE;
long maxEventTime = Long.MIN_VALUE;
for (Map.Entry<String, List<HoodieWriteStat>> entry : partitionToWriteStats.entrySet()) {
for (HoodieWriteStat writeStat : entry.getValue()) {
minEventTime = writeStat.getMinEventTime() != null ? Math.min(writeStat.getMinEventTime(), minEventTime) : minEventTime;
maxEventTime = writeStat.getMaxEventTime() != null ? Math.max(writeStat.getMaxEventTime(), maxEventTime) : maxEventTime;
}
}
return Pair.of(
minEventTime == Long.MAX_VALUE ? Option.empty() : Option.of(minEventTime),
maxEventTime == Long.MIN_VALUE ? Option.empty() : Option.of(maxEventTime));
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
Expand Up @@ -24,9 +24,20 @@
*/
public class HoodiePayloadProps {

// payload ordering field. This could be used to merge incoming record with that in storage. Implementations of
// {@link HoodieRecordPayload} can leverage if required.
/**
* Property for payload ordering field; to be used to merge incoming record with that in storage.
* Implementations of {@link HoodieRecordPayload} can leverage if required.
*
* @see DefaultHoodieRecordPayload
*/
public static final String PAYLOAD_ORDERING_FIELD_PROP = "hoodie.payload.ordering.field";
public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";

/**
* Property for payload event time field; to be used to extract source event time info.
*
* @see DefaultHoodieRecordPayload
*/
public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = "hoodie.payload.event.time.field";
public static String DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL = "ts";
}
Expand Up @@ -143,6 +143,18 @@ public class HoodieWriteStat implements Serializable {
*/
private long fileSizeInBytes;

/**
* The earliest of incoming records' event times (Epoch ms) for calculating latency.
*/
@Nullable
private Long minEventTime;

/**
* The latest of incoming records' event times (Epoch ms) for calculating freshness.
*/
@Nullable
private Long maxEventTime;

@Nullable
@JsonIgnore
private RuntimeStats runtimeStats;
Expand Down Expand Up @@ -303,6 +315,30 @@ public void setFileSizeInBytes(long fileSizeInBytes) {
this.fileSizeInBytes = fileSizeInBytes;
}

public Long getMinEventTime() {
return minEventTime;
}

public void setMinEventTime(Long minEventTime) {
if (this.minEventTime == null) {
this.minEventTime = minEventTime;
} else {
this.minEventTime = Math.min(minEventTime, this.minEventTime);
}
}

public Long getMaxEventTime() {
return maxEventTime;
}

public void setMaxEventTime(Long maxEventTime) {
if (this.maxEventTime == null) {
this.maxEventTime = maxEventTime;
} else {
this.maxEventTime = Math.max(maxEventTime, this.maxEventTime);
}
}

@Nullable
public RuntimeStats getRuntimeStats() {
return runtimeStats;
Expand Down
@@ -0,0 +1,40 @@
/*
* 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.hudi.common.util;

import java.time.Instant;
import java.time.format.DateTimeParseException;
import java.util.Objects;

public class DateTimeUtils {

/**
* Parse input String to a {@link java.time.Instant}.
* @param s Input String should be Epoch time in millisecond or ISO-8601 format.
*/
public static Instant parseDateTime(String s) throws DateTimeParseException {
ValidationUtils.checkArgument(Objects.nonNull(s), "Input String cannot be null.");
try {
return Instant.ofEpochMilli(Long.parseLong(s));
} catch (NumberFormatException e) {
return Instant.parse(s);
}
}
}

0 comments on commit 899ae70

Please sign in to comment.