1
0

capture record metadata before deflating for record counting

This commit is contained in:
Kaushik Devarajaiah
2017-08-22 14:48:04 -07:00
committed by vinoth chandar
parent f2980052cd
commit c98ee057fc
9 changed files with 192 additions and 23 deletions

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieMergeOnReadTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
@@ -224,7 +225,26 @@ public class TestMergeOnReadTable {
assertEquals("Must contain 200 records", 200, readClient.readSince("000").count());
}
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
String newCommitTime = "001";
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus .mergeMetadataForWriteStatuses(statuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * records.size()
assertEquals(String.valueOf(2 * records.size()), allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@Test
public void testSimpleInsertAndDelete() throws Exception {
HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);

View File

@@ -18,9 +18,14 @@ package com.uber.hoodie.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
@@ -91,6 +96,15 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
}
}
@Override
public Optional<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Optional.of(metadataMap);
}
public String getRowKey() {
return rowKey;
}
@@ -120,4 +134,58 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
IOUtils.copy(iis, sw);
return sw.toString();
}
/**
* A custom {@link WriteStatus} that merges passed metadata key value map
* to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
*/
public static class MetadataMergeWriteStatus extends WriteStatus {
private Map<String, String> mergedMetadataMap = new HashMap<>();
@Override
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public void markFailure(HoodieRecord record, Throwable t,
Optional<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if(recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
public static Map<String, String> mergeMetadataForWriteStatuses(List<WriteStatus> writeStatuses) {
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
for (WriteStatus writeStatus : writeStatuses) {
MetadataMergeWriteStatus.mergeMetadataMaps(
((MetadataMergeWriteStatus)writeStatus).getMergedMetadataMap(),
allWriteStatusMergedMetadataMap);
}
return allWriteStatusMergedMetadataMap;
}
private static void mergeMetadataMaps(Map<String, String> mergeFromMap, Map<String, String> mergeToMap) {
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
String key = entry.getKey();
if(!mergeToMap.containsKey(key)) {
mergeToMap.put(key, "0");
}
mergeToMap
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
}
}
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
private static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
}
}

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.table;
import com.uber.hoodie.common.TestRawTripPayload.MetadataMergeWriteStatus;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -34,13 +35,13 @@ import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.config.HoodieStorageConfig;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
@@ -238,7 +239,41 @@ public class TestCopyOnWriteTable {
return records;
}
@Test public void testInsertWithPartialFailures() throws Exception {
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class).build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Get some records belong to the same partition (2016/01/31)
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
List<WriteStatus> writeStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus
.mergeMetadataForWriteStatuses(writeStatuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this should be 2 * 3
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@Test
public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();