1
0

[HUDI-603]: DeltaStreamer can now fetch schema before every run in continuous mode (#1566)

Co-authored-by: Balaji Varadarajan <balaji.varadarajan@robinhood.com>
This commit is contained in:
Pratyaksh Sharma
2020-10-07 09:04:03 +05:30
committed by GitHub
parent fed01cd3c9
commit 524193eb4b
9 changed files with 252 additions and 57 deletions

View File

@@ -24,6 +24,8 @@ import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -51,6 +53,7 @@ import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback;
import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaSet;
import org.apache.hudi.utilities.sources.InputBatch;
import org.apache.hudi.utilities.transform.Transformer;
@@ -166,6 +169,17 @@ public class DeltaSync implements Serializable {
*/
private transient Option<HoodieTimeline> commitTimelineOpt;
/**
* Tracks whether new schema is being seen and creates client accordingly.
*/
private final SchemaSet processedSchema;
/**
* DeltaSync will explicitly manage embedded timeline server so that they can be reused across Write Client
* instantiations.
*/
private transient Option<EmbeddedTimelineService> embeddedTimelineService = Option.empty();
/**
* Write Client.
*/
@@ -184,6 +198,7 @@ public class DeltaSync implements Serializable {
this.onInitializingHoodieWriteClient = onInitializingHoodieWriteClient;
this.props = props;
this.userProvidedSchemaProvider = schemaProvider;
this.processedSchema = new SchemaSet();
refreshTimeline();
// Register User Provided schema first
@@ -244,6 +259,18 @@ public class DeltaSync implements Serializable {
this.schemaProvider = srcRecordsWithCkpt.getKey();
// Setup HoodieWriteClient and compaction now that we decided on schema
setupWriteClient();
} else {
Schema newSourceSchema = srcRecordsWithCkpt.getKey().getSourceSchema();
Schema newTargetSchema = srcRecordsWithCkpt.getKey().getTargetSchema();
if (!(processedSchema.isSchemaPresent(newSourceSchema))
|| !(processedSchema.isSchemaPresent(newTargetSchema))) {
LOG.info("Seeing new schema. Source :" + newSourceSchema.toString(true)
+ ", Target :" + newTargetSchema.toString(true));
// We need to recreate write client with new schema and register them.
reInitWriteClient(newSourceSchema, newTargetSchema);
processedSchema.addSchema(newSourceSchema);
processedSchema.addSchema(newTargetSchema);
}
}
result = writeToSink(srcRecordsWithCkpt.getRight().getRight(),
@@ -541,22 +568,49 @@ public class DeltaSync implements Serializable {
* SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of
* this constraint.
*/
private void setupWriteClient() {
LOG.info("Setting up Hoodie Write Client");
if ((null != schemaProvider) && (null == writeClient)) {
registerAvroSchemas(schemaProvider);
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true);
onInitializingHoodieWriteClient.apply(writeClient);
public void setupWriteClient() throws IOException {
if ((null != schemaProvider)) {
Schema sourceSchema = schemaProvider.getSourceSchema();
Schema targetSchema = schemaProvider.getTargetSchema();
reInitWriteClient(sourceSchema, targetSchema);
}
}
private void reInitWriteClient(Schema sourceSchema, Schema targetSchema) throws IOException {
LOG.info("Setting up new Hoodie Write Client");
registerAvroSchemas(sourceSchema, targetSchema);
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(targetSchema);
if (hoodieCfg.isEmbeddedTimelineServerEnabled()) {
if (!embeddedTimelineService.isPresent()) {
embeddedTimelineService = EmbeddedTimelineServerHelper.createEmbeddedTimelineService(new HoodieSparkEngineContext(jssc), hoodieCfg);
} else {
EmbeddedTimelineServerHelper.updateWriteConfigWithTimelineServer(embeddedTimelineService.get(), hoodieCfg);
}
}
if (null != writeClient) {
// Close Write client.
writeClient.close();
}
writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true, embeddedTimelineService);
onInitializingHoodieWriteClient.apply(writeClient);
}
/**
* Helper to construct Write Client config.
*
* @param schemaProvider Schema Provider
*/
private HoodieWriteConfig getHoodieClientConfig(SchemaProvider schemaProvider) {
return getHoodieClientConfig(schemaProvider != null ? schemaProvider.getTargetSchema() : null);
}
/**
* Helper to construct Write Client config.
*
* @param schema Schema
*/
private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
final boolean combineBeforeUpsert = true;
final boolean autoCommit = false;
HoodieWriteConfig.Builder builder =
@@ -567,8 +621,8 @@ public class DeltaSync implements Serializable {
.forTable(cfg.targetTableName)
.withAutoCommit(autoCommit).withProps(props);
if (null != schemaProvider && null != schemaProvider.getTargetSchema()) {
builder = builder.withSchema(schemaProvider.getTargetSchema().toString());
if (null != schema) {
builder = builder.withSchema(schema.toString());
}
HoodieWriteConfig config = builder.build();
@@ -596,12 +650,24 @@ public class DeltaSync implements Serializable {
* @param schemaProvider Schema Provider
*/
private void registerAvroSchemas(SchemaProvider schemaProvider) {
// register the schemas, so that shuffle does not serialize the full schemas
if (null != schemaProvider) {
registerAvroSchemas(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema());
}
}
/**
* Register Avro Schemas.
*
* @param sourceSchema Source Schema
* @param targetSchema Target Schema
*/
private void registerAvroSchemas(Schema sourceSchema, Schema targetSchema) {
// register the schemas, so that shuffle does not serialize the full schemas
if (null != sourceSchema) {
List<Schema> schemas = new ArrayList<>();
schemas.add(schemaProvider.getSourceSchema());
if (schemaProvider.getTargetSchema() != null) {
schemas.add(schemaProvider.getTargetSchema());
schemas.add(sourceSchema);
if (targetSchema != null) {
schemas.add(targetSchema);
}
LOG.info("Registering Schema :" + schemas);
@@ -617,6 +683,11 @@ public class DeltaSync implements Serializable {
writeClient.close();
writeClient = null;
}
LOG.info("Shutting down embedded timeline server");
if (embeddedTimelineService.isPresent()) {
embeddedTimelineService.get().stop();
}
}
public FileSystem getFs() {

View File

@@ -624,21 +624,26 @@ public class HoodieDeltaStreamer implements Serializable {
*/
protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
if (cfg.isAsyncCompactionEnabled()) {
asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
// Enqueue existing pending compactions first
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
pending.forEach(hoodieInstant -> asyncCompactService.enqueuePendingCompaction(hoodieInstant));
asyncCompactService.start((error) -> {
// Shutdown DeltaSync
shutdown(false);
return true;
});
try {
asyncCompactService.waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
} catch (InterruptedException ie) {
throw new HoodieException(ie);
if (null != asyncCompactService) {
// Update the write client used by Async Compactor.
asyncCompactService.updateWriteClient(writeClient);
} else {
asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
// Enqueue existing pending compactions first
HoodieTableMetaClient meta =
new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
pending.forEach(hoodieInstant -> asyncCompactService.enqueuePendingCompaction(hoodieInstant));
asyncCompactService.start((error) -> {
// Shutdown DeltaSync
shutdown(false);
return true;
});
try {
asyncCompactService.waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
} catch (InterruptedException ie) {
throw new HoodieException(ie);
}
}
}
return true;

View File

@@ -48,9 +48,6 @@ public class SchemaRegistryProvider extends SchemaProvider {
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
}
private final Schema schema;
private final Schema targetSchema;
private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
URL registry = new URL(registryUrl);
ObjectMapper mapper = new ObjectMapper();
@@ -61,18 +58,6 @@ public class SchemaRegistryProvider extends SchemaProvider {
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
String registryUrl = props.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
String targetRegistryUrl = props.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
try {
this.schema = getSchema(registryUrl);
if (!targetRegistryUrl.equals(registryUrl)) {
this.targetSchema = getSchema(targetRegistryUrl);
} else {
this.targetSchema = schema;
}
} catch (IOException ioe) {
throw new HoodieIOException("Error reading schema from registry :" + registryUrl, ioe);
}
}
private static Schema getSchema(String registryUrl) throws IOException {
@@ -81,11 +66,22 @@ public class SchemaRegistryProvider extends SchemaProvider {
@Override
public Schema getSourceSchema() {
return schema;
String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
try {
return getSchema(registryUrl);
} catch (IOException ioe) {
throw new HoodieIOException("Error reading source schema from registry :" + registryUrl, ioe);
}
}
@Override
public Schema getTargetSchema() {
return targetSchema;
String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
String targetRegistryUrl = config.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
try {
return getSchema(targetRegistryUrl);
} catch (IOException ioe) {
throw new HoodieIOException("Error reading target schema from registry :" + registryUrl, ioe);
}
}
}

View File

@@ -0,0 +1,44 @@
/*
* 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.utilities.schema;
import java.io.Serializable;
import java.util.HashSet;
import org.apache.avro.Schema;
import org.apache.avro.SchemaNormalization;
import java.util.Set;
/**
* Tracks already processed schemas.
*/
public class SchemaSet implements Serializable {
private final Set<Long> processedSchema = new HashSet<>();
public boolean isSchemaPresent(Schema schema) {
long schemaKey = SchemaNormalization.parsingFingerprint64(schema);
return processedSchema.contains(schemaKey);
}
public void addSchema(Schema schema) {
long schemaKey = SchemaNormalization.parsingFingerprint64(schema);
processedSchema.add(schemaKey);
}
}