1
0

HUDI-70 : Making DeltaStreamer run in continuous mode with concurrent compaction

This commit is contained in:
Balaji Varadarajan
2019-05-15 13:21:55 -07:00
committed by Balaji Varadarajan
parent 3a210ef08e
commit a0d7ab2384
32 changed files with 2000 additions and 441 deletions

View File

@@ -23,6 +23,7 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -47,13 +48,21 @@ public abstract class AbstractHoodieClient implements Serializable {
* of the cached file-system view. New completed actions will be synced automatically
* in an incremental fashion.
*/
private transient EmbeddedTimelineService timelineServer;
private transient Optional<EmbeddedTimelineService> timelineServer;
private final boolean shouldStopTimelineServer;
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
this(jsc, clientConfig, Optional.empty());
}
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
Optional<EmbeddedTimelineService> timelineServer) {
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
this.jsc = jsc;
this.basePath = clientConfig.getBasePath();
this.config = clientConfig;
this.timelineServer = timelineServer;
shouldStopTimelineServer = !timelineServer.isPresent();
startEmbeddedServerView();
}
@@ -65,28 +74,30 @@ public abstract class AbstractHoodieClient implements Serializable {
}
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
if (timelineServer != null) {
if (timelineServer.isPresent() && shouldStopTimelineServer) {
// Stop only if owner
logger.info("Stopping Timeline service !!");
timelineServer.stop();
timelineServer = null;
// Reset Storage Config to Client specified config
if (resetViewStorageConfig) {
config.resetViewStorageConfig();
}
timelineServer.get().stop();
}
timelineServer = Optional.empty();
// Reset Storage Config to Client specified config
if (resetViewStorageConfig) {
config.resetViewStorageConfig();
}
}
private synchronized void startEmbeddedServerView() {
if (config.isEmbeddedTimelineServerEnabled()) {
if (timelineServer == null) {
if (!timelineServer.isPresent()) {
// Run Embedded Timeline Server
logger.info("Starting Timeline service !!");
timelineServer = new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
config.getClientSpecifiedViewStorageConfig());
timelineServer = Optional.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
config.getClientSpecifiedViewStorageConfig()));
try {
timelineServer.startServer();
timelineServer.get().startServer();
// Allow executor to find this newly instantiated timeline service
config.setViewStorageConfig(timelineServer.getRemoteFileSystemViewConfig());
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
} catch (IOException e) {
logger.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
stopEmbeddedServerView(false);
@@ -98,4 +109,12 @@ public abstract class AbstractHoodieClient implements Serializable {
logger.info("Embedded Timeline Server is disabled. Not starting timeline service");
}
}
public HoodieWriteConfig getConfig() {
return config;
}
public Optional<EmbeddedTimelineService> getTimelineServer() {
return timelineServer;
}
}

View File

@@ -24,6 +24,7 @@ import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.model.CompactionOperation;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieDataFile;
@@ -68,6 +69,11 @@ public class CompactionAdminClient extends AbstractHoodieClient {
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
}
public CompactionAdminClient(JavaSparkContext jsc, String basePath,
java.util.Optional<EmbeddedTimelineService> timelineServer) {
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build(), timelineServer);
}
/**
* Validate all compaction operations in a compaction plan. Verifies the file-slices are consistent with corresponding
* compaction operations.

View File

@@ -20,6 +20,7 @@ package com.uber.hoodie;
import com.google.common.base.Optional;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -69,12 +70,20 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
/**
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
public HoodieReadClient(JavaSparkContext jsc, String basePath,
java.util.Optional<EmbeddedTimelineService> timelineService) {
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
// by default we use HoodieBloomIndex
.withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build());
.build(), timelineService);
}
/**
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this(jsc, basePath, java.util.Optional.empty());
}
/**
@@ -91,13 +100,19 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
* @param clientConfig instance of HoodieWriteConfig
*/
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
super(jsc, clientConfig);
this(jsc, clientConfig, java.util.Optional.empty());
}
/**
* @param clientConfig instance of HoodieWriteConfig
*/
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
java.util.Optional<EmbeddedTimelineService> timelineService) {
super(jsc, clientConfig, timelineService);
final String basePath = clientConfig.getBasePath();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
this.hoodieTable = HoodieTable
.getHoodieTable(metaClient,
clientConfig, jsc);
this.hoodieTable = HoodieTable.getHoodieTable(metaClient, clientConfig, jsc);
this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
this.index = HoodieIndex.createIndex(clientConfig, jsc);
this.sqlContextOpt = Optional.absent();

View File

@@ -27,6 +27,7 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieRollbackStat;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
@@ -75,6 +76,7 @@ import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
@@ -124,7 +126,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
@VisibleForTesting
HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
boolean rollbackInFlight, HoodieIndex index) {
super(jsc, clientConfig);
this(jsc, clientConfig, rollbackInFlight, index, Optional.empty());
}
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
boolean rollbackInFlight, HoodieIndex index, Optional<EmbeddedTimelineService> timelineService) {
super(jsc, clientConfig, timelineService);
this.index = index;
this.metrics = new HoodieMetrics(config, config.getTableName());
this.rollbackInFlight = rollbackInFlight;
@@ -1184,7 +1191,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
private HoodieTable getTableAndInitCtx(JavaRDD<HoodieRecord<T>> records) {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
new HoodieTableMetaClient(
// Clone Configuration here. Otherwise we could see ConcurrentModificationException (race) in multi-threaded
// execution (HoodieDeltaStreamer) when Configuration gets serialized by Spark.
new Configuration(jsc.hadoopConfiguration()), config.getBasePath(), true), config, jsc);
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
writeContext = metrics.getCommitCtx();
} else {

View File

@@ -91,6 +91,10 @@ public class EmbeddedTimelineService {
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
}
public FileSystemViewManager getViewManager() {
return viewManager;
}
public void stop() {
if (null != server) {
this.server.close();

View File

@@ -31,16 +31,23 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
@@ -83,17 +90,23 @@ public class HoodieTestDataGenerator {
private static Random rand = new Random(46474747);
private List<KeyPartition> existingKeysList = new ArrayList<>();
private String[] partitionPaths;
private final Map<Integer, KeyPartition> existingKeys;
private final String[] partitionPaths;
private int numExistingKeys;
public HoodieTestDataGenerator(String[] partitionPaths) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this(partitionPaths, new HashMap<>());
}
public HoodieTestDataGenerator() {
this(DEFAULT_PARTITION_PATHS);
}
public HoodieTestDataGenerator(String[] partitionPaths, Map<Integer, KeyPartition> keyPartitionMap) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeys = keyPartitionMap;
}
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
for (String partitionPath : partitionPaths) {
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0);
@@ -193,19 +206,29 @@ public class HoodieTestDataGenerator {
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, Integer n) throws IOException {
List<HoodieRecord> inserts = new ArrayList<>();
for (int i = 0; i < n; i++) {
return generateInsertsStream(commitTime, n).collect(Collectors.toList());
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n) {
int currSize = getNumExistingKeys();
return IntStream.range(0, n).boxed().map(i -> {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
}
return inserts;
existingKeys.put(currSize + i, kp);
numExistingKeys++;
try {
return new HoodieRecord(key, generateRandomValue(key, commitTime));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
public List<HoodieRecord> generateSameKeyInserts(String commitTime, List<HoodieRecord> origin) throws IOException {
@@ -221,6 +244,7 @@ public class HoodieTestDataGenerator {
public List<HoodieRecord> generateInsertsWithHoodieAvroPayload(String commitTime, int limit) throws
IOException {
List<HoodieRecord> inserts = new ArrayList<>();
int currSize = getNumExistingKeys();
for (int i = 0; i < limit; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
@@ -230,7 +254,8 @@ public class HoodieTestDataGenerator {
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeysList.add(kp);
existingKeys.put(currSize + i, kp);
numExistingKeys++;
}
return inserts;
}
@@ -293,7 +318,7 @@ public class HoodieTestDataGenerator {
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
KeyPartition kp = existingKeys.get(rand.nextInt(numExistingKeys - 1));
HoodieRecord record = generateUpdateRecord(kp.key, commitTime);
updates.add(record);
}
@@ -307,39 +332,55 @@ public class HoodieTestDataGenerator {
* @param n Number of unique records
* @return list of hoodie record updates
*/
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
Set<KeyPartition> used = new HashSet<>();
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) {
return generateUniqueUpdatesStream(commitTime, n).collect(Collectors.toList());
}
if (n > existingKeysList.size()) {
/**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
* @param commitTime Commit Timestamp
* @param n Number of unique records
* @return stream of hoodie record updates
*/
public Stream<HoodieRecord> generateUniqueUpdatesStream(String commitTime, Integer n) {
final Set<KeyPartition> used = new HashSet<>();
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique updates is greater than number of available keys");
}
for (int i = 0; i < n; i++) {
int index = rand.nextInt(existingKeysList.size() - 1);
KeyPartition kp = existingKeysList.get(index);
return IntStream.range(0, n).boxed().map(i -> {
int index = numExistingKeys == 1 ? 0 : rand.nextInt(numExistingKeys - 1);
KeyPartition kp = existingKeys.get(index);
// Find the available keyPartition starting from randomly chosen one.
while (used.contains(kp)) {
index = (index + 1) % existingKeysList.size();
kp = existingKeysList.get(index);
index = (index + 1) % numExistingKeys;
kp = existingKeys.get(index);
}
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
updates.add(record);
used.add(kp);
}
return updates;
try {
return new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
public String[] getPartitionPaths() {
return partitionPaths;
}
public List<KeyPartition> getExistingKeysList() {
return existingKeysList;
public int getNumExistingKeys() {
return numExistingKeys;
}
public static class KeyPartition {
public static class KeyPartition implements Serializable {
HoodieKey key;
String partitionPath;
}
public void close() {
existingKeys.clear();
}
}

View File

@@ -905,6 +905,7 @@ public class TestMergeOnReadTable {
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))
.count() > 0);
}
writeClient.close();
}
@Test