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-1637] Add writer, operation, and partition info to failed metadata writer events #3498

Merged
merged 3 commits into from
May 5, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
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 @@ -26,12 +26,16 @@
import com.google.common.util.concurrent.ListenableFuture;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;

import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.avro.Schema;
Expand Down Expand Up @@ -114,8 +118,12 @@ public void flush(String dbName, String tableName) throws IOException {
for (HashMap.Entry<List<String>, ListenableFuture<Void>> execution : executionMap.entrySet()) {
try {
execution.getValue().get(timeOutSeconds, TimeUnit.SECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
throw new RuntimeException("Error when getting the result of registration for table" + tableKey, e);
} catch (TimeoutException e) {
// Since TimeoutException should always be a transient issue, throw RuntimeException which will fail/retry container
throw new RuntimeException("Timeout waiting for result of registration for table " + tableKey, e);
} catch (InterruptedException | ExecutionException e) {
Set<String> partitions = executionMap.keySet().stream().flatMap(List::stream).collect(Collectors.toSet());
throw new HiveMetadataWriterWithPartitionInfoException(partitions, Collections.emptySet(), e);
}
}
executionMap.clear();
Expand Down Expand Up @@ -243,7 +251,7 @@ private void registerSpec(String tableKey, List<String> partitionValue, HiveSpec
try {
executionMap.get(partitionValue).get(timeOutSeconds, TimeUnit.SECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
log.error("Error when getting the result of registration for table" + tableKey);
log.error("Error when getting the result of registration for table " + tableKey);
throw new RuntimeException(e);
}
}
Expand Down Expand Up @@ -326,13 +334,27 @@ public void writeEnvelope(RecordEnvelope<GenericRecord> recordEnvelope, Map<Stri
GobblinMetadataChangeEvent gmce =
(GobblinMetadataChangeEvent) SpecificData.get().deepCopy(genericRecord.getSchema(), genericRecord);
if (whitelistBlacklist.acceptTable(tableSpec.getTable().getDbName(), tableSpec.getTable().getTableName())) {
write(gmce, newSpecsMap, oldSpecsMap, tableSpec);
try {
write(gmce, newSpecsMap, oldSpecsMap, tableSpec);
} catch (IOException e) {
throw new HiveMetadataWriterWithPartitionInfoException(getPartitionValues(newSpecsMap), getPartitionValues(oldSpecsMap), e);
}
} else {
log.debug(String.format("Skip table %s.%s since it's not selected", tableSpec.getTable().getDbName(),
tableSpec.getTable().getTableName()));
}
}

/**
* Extract a unique list of partition values as strings from a map of HiveSpecs.
*/
public Set<String> getPartitionValues(Map<String, Collection<HiveSpec>> specMap) {
Set<HiveSpec> hiveSpecs = specMap.values().stream().flatMap(Collection::stream).collect(Collectors.toSet());
Set<List<String>> partitionValueLists = hiveSpecs.stream().filter(spec -> spec.getPartition().isPresent())
.map(spec -> spec.getPartition().get().getValues()).collect(Collectors.toSet());
return partitionValueLists.stream().flatMap(List::stream).collect(Collectors.toSet());
}

@Override
public void close() throws IOException {
this.closer.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.hive.writer;

import java.io.IOException;
import java.util.Set;


public class HiveMetadataWriterWithPartitionInfoException extends IOException {
public Set<String> addedPartitionValues;
public Set<String> droppedPartitionValues;

HiveMetadataWriterWithPartitionInfoException(Set<String> addedPartitionValues, Set<String> droppedPartitionValues, Exception exception) {
super(exception);
this.addedPartitionValues = addedPartitionValues;
this.droppedPartitionValues = droppedPartitionValues;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,11 @@
import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.configuration.State;
import org.apache.gobblin.dataset.Descriptor;
import org.apache.gobblin.hive.HiveRegistrationUnit;
import org.apache.gobblin.hive.policy.HiveRegistrationPolicy;
import org.apache.gobblin.hive.policy.HiveRegistrationPolicyBase;
import org.apache.gobblin.hive.spec.HiveSpec;
import org.apache.gobblin.hive.writer.HiveMetadataWriterWithPartitionInfoException;
import org.apache.gobblin.hive.writer.MetadataWriter;
import org.apache.gobblin.instrumented.Instrumented;
import org.apache.gobblin.metadata.DataFile;
Expand Down Expand Up @@ -109,13 +111,14 @@ public class GobblinMCEWriter implements DataWriter<GenericRecord> {
List<MetadataWriter> metadataWriters;
Map<String, TableStatus> tableOperationTypeMap;
@Getter
Map<String, Map<String, GobblinMetadataException>> datasetErrorMap;
Map<String, Map<String, List<GobblinMetadataException>>> datasetErrorMap;
Set<String> acceptedClusters;
protected State state;
private final ParallelRunner parallelRunner;
private int parallelRunnerTimeoutMills;
private Map<String, Cache<String, Collection<HiveSpec>>> oldSpecsMaps;
private Map<String, Cache<String, Collection<HiveSpec>>> newSpecsMaps;
private Map<String, List<HiveRegistrationUnit.Column>> partitionKeysMap;
private Closer closer = Closer.create();
protected final AtomicLong recordCount = new AtomicLong(0L);
@Setter
Expand All @@ -136,6 +139,7 @@ static class TableStatus {
oldSpecsMaps = new HashMap<>();
metadataWriters = new ArrayList<>();
datasetErrorMap = new HashMap<>();
partitionKeysMap = new HashMap<>();
acceptedClusters = properties.getPropAsSet(ACCEPTED_CLUSTER_NAMES, ClustersNames.getInstance().getClusterName());
state = properties;
maxErrorDataset = state.getPropAsInt(GMCE_METADATA_WRITER_MAX_ERROR_DATASET, DEFUALT_GMCE_METADATA_WRITER_MAX_ERROR_DATASET);
Expand Down Expand Up @@ -281,6 +285,7 @@ public void writeEnvelope(RecordEnvelope<GenericRecord> recordEnvelope) throws I
String dbName = spec.getTable().getDbName();
String tableName = spec.getTable().getTableName();
String tableString = Joiner.on(TABLE_NAME_DELIMITER).join(dbName, tableName);
partitionKeysMap.put(tableString, spec.getTable().getPartitionKeys());
if (!tableOperationTypeMap.containsKey(tableString)) {
tableOperationTypeMap.put(tableString, new TableStatus(gmce.getOperationType(),
gmce.getDatasetIdentifier().getNativeName(), watermark.getSource(),
Expand Down Expand Up @@ -331,7 +336,7 @@ void writeWithMetadataWriters(
} catch (Exception e) {
meetException = true;
writer.reset(dbName, tableName);
addOrThrowException(e, tableString, dbName, tableName);
addOrThrowException(e, tableString, dbName, tableName, getFailedWriterList(writer));
}
}
}
Expand All @@ -355,15 +360,26 @@ static List<MetadataWriter> getAllowedMetadataWriters(GobblinMetadataChangeEvent
.collect(Collectors.toList());
}

private void addOrThrowException(Exception e, String tableString, String dbName, String tableName) throws IOException{
private void addOrThrowException(Exception e, String tableString, String dbName, String tableName, List<String> failedWriters) throws IOException {
TableStatus tableStatus = tableOperationTypeMap.get(tableString);
Map<String, GobblinMetadataException> tableErrorMap = this.datasetErrorMap.getOrDefault(tableStatus.datasetPath, new HashMap<>());
if (tableErrorMap.containsKey(tableString)) {
tableErrorMap.get(tableString).highWatermark = tableStatus.gmceHighWatermark;
Map<String, List<GobblinMetadataException>> tableErrorMap = this.datasetErrorMap.getOrDefault(tableStatus.datasetPath, new HashMap<>());
GobblinMetadataException lastException = null;
if (tableErrorMap.containsKey(tableString) && !tableErrorMap.get(tableString).isEmpty()) {
lastException = tableErrorMap.get(tableString).get(tableErrorMap.get(tableString).size() - 1);
} else {
GobblinMetadataException gobblinMetadataException =
new GobblinMetadataException(tableStatus.datasetPath, dbName, tableName, tableStatus.gmceTopicPartition, tableStatus.gmceLowWatermark, tableStatus.gmceHighWatermark, e);
tableErrorMap.put(tableString, gobblinMetadataException);
tableErrorMap.put(tableString, new ArrayList<>());
}
// If operationType has changed, add a new exception to the list so that each failure event represents an offset range all containing the same operation
if (lastException != null && lastException.operationType.equals(tableStatus.operationType)) {
lastException.highWatermark = tableStatus.gmceHighWatermark;
} else {
lastException = new GobblinMetadataException(tableStatus.datasetPath, dbName, tableName, tableStatus.gmceTopicPartition,
tableStatus.gmceLowWatermark, tableStatus.gmceHighWatermark, failedWriters, tableStatus.operationType, partitionKeysMap.get(tableString), e);
tableErrorMap.get(tableString).add(lastException);
}
if (e instanceof HiveMetadataWriterWithPartitionInfoException) {
lastException.addedPartitionValues.addAll(((HiveMetadataWriterWithPartitionInfoException) e).addedPartitionValues);
lastException.droppedPartitionValues.addAll(((HiveMetadataWriterWithPartitionInfoException) e).droppedPartitionValues);
}
this.datasetErrorMap.put(tableStatus.datasetPath, tableErrorMap);
tableOperationTypeMap.remove(tableString);
Expand Down Expand Up @@ -391,18 +407,20 @@ private void flush(String dbName, String tableName) throws IOException {
} catch (IOException e) {
meetException = true;
writer.reset(dbName, tableName);
addOrThrowException(e, tableString, dbName, tableName);
addOrThrowException(e, tableString, dbName, tableName, getFailedWriterList(writer));
}
}
}
String datasetPath = tableOperationTypeMap.get(tableString).datasetPath;
if (!meetException && datasetErrorMap.containsKey(datasetPath) && datasetErrorMap.get(datasetPath).containsKey(tableString)) {
// We only want to emit GTE when the table watermark moves. There can be two scenario that watermark move, one is after one flush interval,
// we commit new watermark to state store, anther is here, where during the flush interval, we flush table because table operation changes.
// Under this condition, error map contains this dataset means we met error before this flush, but this time when flush succeed and
// the watermark inside the table moves, so we want to emit GTE to indicate there is some data loss here
submitFailureEvent(datasetErrorMap.get(datasetPath).get(tableString));
this.datasetErrorMap.get(datasetPath).remove(tableString);
if (!meetException) {
String datasetPath = tableOperationTypeMap.get(tableString).datasetPath;
if (datasetErrorMap.containsKey(datasetPath) && datasetErrorMap.get(datasetPath).containsKey(tableString)) {
// We only want to emit GTE when the table watermark moves. There can be two scenario that watermark move, one is after one flush interval,
// we commit new watermark to state store, anther is here, where during the flush interval, we flush table because table operation changes.
// Under this condition, error map contains this dataset means we met error before this flush, but this time when flush succeed and
// the watermark inside the table moves, so we want to emit GTE to indicate there is some data loss here
submitFailureEvents(datasetErrorMap.get(datasetPath).get(tableString));
this.datasetErrorMap.get(datasetPath).remove(tableString);
}
}
}

Expand All @@ -428,9 +446,9 @@ public void flush() throws IOException {
tableOperationTypeMap.clear();
recordCount.lazySet(0L);
// Emit events for all current errors, since the GMCE watermark will be advanced
for (Map.Entry<String, Map<String, GobblinMetadataException>> entry : datasetErrorMap.entrySet()) {
for (GobblinMetadataException exception : entry.getValue().values()) {
submitFailureEvent(exception);
for (Map.Entry<String, Map<String, List<GobblinMetadataException>>> entry : datasetErrorMap.entrySet()) {
for (List<GobblinMetadataException> exceptionList : entry.getValue().values()) {
submitFailureEvents(exceptionList);
}
entry.getValue().clear();
}
Expand Down Expand Up @@ -483,23 +501,40 @@ public static State setHiveRegProperties(State state, GobblinMetadataChangeEvent
}

/**
* Submit event indicating that a specific set of GMCEs have been skipped, so there is a gap in the registration
* Submit events indicating that a specific set of GMCEs have been skipped, so there is a gap in the registration
*/
private void submitFailureEvent(GobblinMetadataException exception) {
log.warn(String.format("Sending GTE to indicate table flush failure for %s.%s", exception.dbName, exception.tableName));

GobblinEventBuilder gobblinTrackingEvent = new GobblinEventBuilder(IcebergMCEMetadataKeys.METADATA_WRITER_FAILURE_EVENT);

gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.DATASET_HDFS_PATH, exception.datasetPath);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.FAILURE_EVENT_DB_NAME, exception.dbName);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.FAILURE_EVENT_TABLE_NAME, exception.tableName);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_TOPIC_NAME, exception.GMCETopicPartition.split("-")[0]);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_TOPIC_PARTITION, exception.GMCETopicPartition.split("-")[1]);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_HIGH_WATERMARK, Long.toString(exception.highWatermark));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_LOW_WATERMARK, Long.toString(exception.lowWatermark));
String message = exception.getCause() == null ? exception.getMessage() : exception.getCause().getMessage();
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.EXCEPTION_MESSAGE_KEY_NAME, message);

eventSubmitter.submit(gobblinTrackingEvent);
private void submitFailureEvents(List<GobblinMetadataException> exceptionList) {
if (exceptionList.isEmpty()) {
return;
}
log.warn(String.format("Sending GTEs to indicate table flush failure for %s.%s", exceptionList.get(0).dbName, exceptionList.get(0).tableName));

for (GobblinMetadataException exception : exceptionList) {
GobblinEventBuilder gobblinTrackingEvent = new GobblinEventBuilder(IcebergMCEMetadataKeys.METADATA_WRITER_FAILURE_EVENT);

gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.DATASET_HDFS_PATH, exception.datasetPath);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.FAILURE_EVENT_DB_NAME, exception.dbName);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.FAILURE_EVENT_TABLE_NAME, exception.tableName);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_TOPIC_NAME, exception.GMCETopicPartition.split("-")[0]);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_TOPIC_PARTITION, exception.GMCETopicPartition.split("-")[1]);
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_HIGH_WATERMARK, Long.toString(exception.highWatermark));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.GMCE_LOW_WATERMARK, Long.toString(exception.lowWatermark));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.FAILED_WRITERS_KEY, Joiner.on(',').join(exception.failedWriters));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.OPERATION_TYPE_KEY, exception.operationType.toString());
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.ADDED_PARTITION_VALUES_KEY, Joiner.on(',').join(exception.addedPartitionValues));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.DROPPED_PARTITION_VALUES_KEY, Joiner.on(',').join(exception.droppedPartitionValues));
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.PARTITION_KEYS, Joiner.on(',').join(exception.partitionKeys.stream()
.map(HiveRegistrationUnit.Column::getName).collect(Collectors.toList())));

String message = exception.getCause() == null ? exception.getMessage() : exception.getCause().getMessage();
gobblinTrackingEvent.addMetadata(IcebergMCEMetadataKeys.EXCEPTION_MESSAGE_KEY_NAME, message);

eventSubmitter.submit(gobblinTrackingEvent);
}
}

private List<String> getFailedWriterList(MetadataWriter failedWriter) {
List<MetadataWriter> failedWriters = metadataWriters.subList(metadataWriters.indexOf(failedWriter), metadataWriters.size());
return failedWriters.stream().map(writer -> writer.getClass().getName()).collect(Collectors.toList());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,12 @@
package org.apache.gobblin.iceberg.writer;

import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;

import org.apache.gobblin.hive.HiveRegistrationUnit;
import org.apache.gobblin.metadata.OperationType;


public class GobblinMetadataException extends IOException {
Expand All @@ -27,14 +33,25 @@ public class GobblinMetadataException extends IOException {
public String GMCETopicPartition;
public long highWatermark;
public long lowWatermark;
public Exception exception;
GobblinMetadataException(String datasetPath, String dbName, String tableName, String GMCETopicPartition, long lowWatermark, long highWatermark, Exception exception) {
public List<String> failedWriters;
public OperationType operationType;
public Set<String> addedPartitionValues;
public Set<String> droppedPartitionValues;
public List<HiveRegistrationUnit.Column> partitionKeys;

GobblinMetadataException(String datasetPath, String dbName, String tableName, String GMCETopicPartition, long lowWatermark, long highWatermark,
List<String> failedWriters, OperationType operationType, List<HiveRegistrationUnit.Column> partitionKeys, Exception exception) {
super(String.format("failed to flush table %s, %s", dbName, tableName), exception);
this.datasetPath = datasetPath;
this.dbName = dbName;
this.tableName = tableName;
this.GMCETopicPartition = GMCETopicPartition;
this.highWatermark = highWatermark;
this.lowWatermark = lowWatermark;
this.failedWriters = failedWriters;
this.operationType = operationType;
this.addedPartitionValues = new HashSet<>();
this.droppedPartitionValues = new HashSet<>();
this.partitionKeys = partitionKeys;
}
}
Loading