[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:
@@ -161,4 +161,8 @@ public abstract class AsyncCompactService extends HoodieAsyncService {
|
|||||||
protected boolean shouldStopCompactor() {
|
protected boolean shouldStopCompactor() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public synchronized void updateWriteClient(AbstractHoodieWriteClient writeClient) {
|
||||||
|
this.compactor.updateWriteClient(writeClient);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -38,4 +38,9 @@ public abstract class AbstractCompactor<T extends HoodieRecordPayload, I, K, O>
|
|||||||
}
|
}
|
||||||
|
|
||||||
public abstract void compact(HoodieInstant instant) throws IOException;
|
public abstract void compact(HoodieInstant instant) throws IOException;
|
||||||
|
|
||||||
|
public void updateWriteClient(AbstractHoodieWriteClient<T, I, K, O> writeClient) {
|
||||||
|
this.compactionClient = writeClient;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,9 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.client;
|
package org.apache.hudi.client;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper;
|
||||||
|
|
||||||
import org.apache.hudi.client.common.EngineProperty;
|
|
||||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
@@ -29,6 +27,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
|||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -100,14 +99,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
|
|||||||
if (config.isEmbeddedTimelineServerEnabled()) {
|
if (config.isEmbeddedTimelineServerEnabled()) {
|
||||||
if (!timelineServer.isPresent()) {
|
if (!timelineServer.isPresent()) {
|
||||||
// Run Embedded Timeline Server
|
// Run Embedded Timeline Server
|
||||||
LOG.info("Starting Timeline service !!");
|
|
||||||
Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
|
|
||||||
timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null),
|
|
||||||
config.getEmbeddedTimelineServerPort(), config.getClientSpecifiedViewStorageConfig()));
|
|
||||||
try {
|
try {
|
||||||
timelineServer.get().startServer();
|
timelineServer = EmbeddedTimelineServerHelper.createEmbeddedTimelineService(context, config);
|
||||||
// Allow executor to find this newly instantiated timeline service
|
|
||||||
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
|
LOG.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
|
||||||
stopEmbeddedServerView(false);
|
stopEmbeddedServerView(false);
|
||||||
@@ -129,4 +122,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
|
|||||||
config.getConsistencyGuardConfig(),
|
config.getConsistencyGuardConfig(),
|
||||||
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
|
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Option<EmbeddedTimelineService> getTimelineServer() {
|
||||||
|
return timelineServer;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -0,0 +1,72 @@
|
|||||||
|
/*
|
||||||
|
* 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.client.embedded;
|
||||||
|
|
||||||
|
import org.apache.hudi.client.common.EngineProperty;
|
||||||
|
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class to instantiate embedded timeline service.
|
||||||
|
*/
|
||||||
|
public class EmbeddedTimelineServerHelper {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(EmbeddedTimelineService.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Instantiate Embedded Timeline Server.
|
||||||
|
* @param context Hoodie Engine Context
|
||||||
|
* @param config Hoodie Write Config
|
||||||
|
* @return TimelineServer if configured to run
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static Option<EmbeddedTimelineService> createEmbeddedTimelineService(
|
||||||
|
HoodieEngineContext context, HoodieWriteConfig config) throws IOException {
|
||||||
|
Option<EmbeddedTimelineService> timelineServer = Option.empty();
|
||||||
|
if (config.isEmbeddedTimelineServerEnabled()) {
|
||||||
|
// Run Embedded Timeline Server
|
||||||
|
LOG.info("Starting Timeline service !!");
|
||||||
|
Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
|
||||||
|
timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null),
|
||||||
|
config.getEmbeddedTimelineServerPort(), config.getClientSpecifiedViewStorageConfig()));
|
||||||
|
timelineServer.get().startServer();
|
||||||
|
updateWriteConfigWithTimelineServer(timelineServer.get(), config);
|
||||||
|
}
|
||||||
|
return timelineServer;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adjusts hoodie write config with timeline server settings.
|
||||||
|
* @param timelineServer Embedded Timeline Server
|
||||||
|
* @param config Hoodie Write Config
|
||||||
|
*/
|
||||||
|
public static void updateWriteConfigWithTimelineServer(EmbeddedTimelineService timelineServer,
|
||||||
|
HoodieWriteConfig config) {
|
||||||
|
// Allow executor to find this newly instantiated timeline service
|
||||||
|
if (config.isEmbeddedTimelineServerEnabled()) {
|
||||||
|
config.setViewStorageConfig(timelineServer.getRemoteFileSystemViewConfig());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -41,7 +41,8 @@ public class HoodieSparkCompactor<T extends HoodieRecordPayload> extends Abstrac
|
|||||||
@Override
|
@Override
|
||||||
public void compact(HoodieInstant instant) throws IOException {
|
public void compact(HoodieInstant instant) throws IOException {
|
||||||
LOG.info("Compactor executing compaction " + instant);
|
LOG.info("Compactor executing compaction " + instant);
|
||||||
JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
|
SparkRDDWriteClient<T> writeClient = (SparkRDDWriteClient<T>)compactionClient;
|
||||||
|
JavaRDD<WriteStatus> res = writeClient.compact(instant.getTimestamp());
|
||||||
long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count();
|
long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count();
|
||||||
if (numWriteErrors != 0) {
|
if (numWriteErrors != 0) {
|
||||||
// We treat even a single error in compaction as fatal
|
// We treat even a single error in compaction as fatal
|
||||||
@@ -50,6 +51,6 @@ public class HoodieSparkCompactor<T extends HoodieRecordPayload> extends Abstrac
|
|||||||
"Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
"Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
||||||
}
|
}
|
||||||
// Commit compaction
|
// Commit compaction
|
||||||
compactionClient.commitCompaction(instant.getTimestamp(), res, Option.empty());
|
writeClient.commitCompaction(instant.getTimestamp(), res, Option.empty());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,6 +24,8 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
|||||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
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.config.TypedProperties;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
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.callback.kafka.HoodieWriteCommitKafkaCallbackConfig;
|
||||||
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
|
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
|
||||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
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.sources.InputBatch;
|
||||||
import org.apache.hudi.utilities.transform.Transformer;
|
import org.apache.hudi.utilities.transform.Transformer;
|
||||||
|
|
||||||
@@ -166,6 +169,17 @@ public class DeltaSync implements Serializable {
|
|||||||
*/
|
*/
|
||||||
private transient Option<HoodieTimeline> commitTimelineOpt;
|
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.
|
* Write Client.
|
||||||
*/
|
*/
|
||||||
@@ -184,6 +198,7 @@ public class DeltaSync implements Serializable {
|
|||||||
this.onInitializingHoodieWriteClient = onInitializingHoodieWriteClient;
|
this.onInitializingHoodieWriteClient = onInitializingHoodieWriteClient;
|
||||||
this.props = props;
|
this.props = props;
|
||||||
this.userProvidedSchemaProvider = schemaProvider;
|
this.userProvidedSchemaProvider = schemaProvider;
|
||||||
|
this.processedSchema = new SchemaSet();
|
||||||
|
|
||||||
refreshTimeline();
|
refreshTimeline();
|
||||||
// Register User Provided schema first
|
// Register User Provided schema first
|
||||||
@@ -244,6 +259,18 @@ public class DeltaSync implements Serializable {
|
|||||||
this.schemaProvider = srcRecordsWithCkpt.getKey();
|
this.schemaProvider = srcRecordsWithCkpt.getKey();
|
||||||
// Setup HoodieWriteClient and compaction now that we decided on schema
|
// Setup HoodieWriteClient and compaction now that we decided on schema
|
||||||
setupWriteClient();
|
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(),
|
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
|
* SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of
|
||||||
* this constraint.
|
* this constraint.
|
||||||
*/
|
*/
|
||||||
private void setupWriteClient() {
|
public void setupWriteClient() throws IOException {
|
||||||
LOG.info("Setting up Hoodie Write Client");
|
if ((null != schemaProvider)) {
|
||||||
if ((null != schemaProvider) && (null == writeClient)) {
|
Schema sourceSchema = schemaProvider.getSourceSchema();
|
||||||
registerAvroSchemas(schemaProvider);
|
Schema targetSchema = schemaProvider.getTargetSchema();
|
||||||
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
|
reInitWriteClient(sourceSchema, targetSchema);
|
||||||
writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true);
|
|
||||||
onInitializingHoodieWriteClient.apply(writeClient);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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.
|
* Helper to construct Write Client config.
|
||||||
*
|
*
|
||||||
* @param schemaProvider Schema Provider
|
* @param schemaProvider Schema Provider
|
||||||
*/
|
*/
|
||||||
private HoodieWriteConfig getHoodieClientConfig(SchemaProvider schemaProvider) {
|
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 combineBeforeUpsert = true;
|
||||||
final boolean autoCommit = false;
|
final boolean autoCommit = false;
|
||||||
HoodieWriteConfig.Builder builder =
|
HoodieWriteConfig.Builder builder =
|
||||||
@@ -567,8 +621,8 @@ public class DeltaSync implements Serializable {
|
|||||||
.forTable(cfg.targetTableName)
|
.forTable(cfg.targetTableName)
|
||||||
.withAutoCommit(autoCommit).withProps(props);
|
.withAutoCommit(autoCommit).withProps(props);
|
||||||
|
|
||||||
if (null != schemaProvider && null != schemaProvider.getTargetSchema()) {
|
if (null != schema) {
|
||||||
builder = builder.withSchema(schemaProvider.getTargetSchema().toString());
|
builder = builder.withSchema(schema.toString());
|
||||||
}
|
}
|
||||||
HoodieWriteConfig config = builder.build();
|
HoodieWriteConfig config = builder.build();
|
||||||
|
|
||||||
@@ -596,12 +650,24 @@ public class DeltaSync implements Serializable {
|
|||||||
* @param schemaProvider Schema Provider
|
* @param schemaProvider Schema Provider
|
||||||
*/
|
*/
|
||||||
private void registerAvroSchemas(SchemaProvider schemaProvider) {
|
private void registerAvroSchemas(SchemaProvider schemaProvider) {
|
||||||
// register the schemas, so that shuffle does not serialize the full schemas
|
|
||||||
if (null != schemaProvider) {
|
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<>();
|
List<Schema> schemas = new ArrayList<>();
|
||||||
schemas.add(schemaProvider.getSourceSchema());
|
schemas.add(sourceSchema);
|
||||||
if (schemaProvider.getTargetSchema() != null) {
|
if (targetSchema != null) {
|
||||||
schemas.add(schemaProvider.getTargetSchema());
|
schemas.add(targetSchema);
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Registering Schema :" + schemas);
|
LOG.info("Registering Schema :" + schemas);
|
||||||
@@ -617,6 +683,11 @@ public class DeltaSync implements Serializable {
|
|||||||
writeClient.close();
|
writeClient.close();
|
||||||
writeClient = null;
|
writeClient = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
LOG.info("Shutting down embedded timeline server");
|
||||||
|
if (embeddedTimelineService.isPresent()) {
|
||||||
|
embeddedTimelineService.get().stop();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public FileSystem getFs() {
|
public FileSystem getFs() {
|
||||||
|
|||||||
@@ -624,6 +624,10 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
*/
|
*/
|
||||||
protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
|
protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
|
||||||
if (cfg.isAsyncCompactionEnabled()) {
|
if (cfg.isAsyncCompactionEnabled()) {
|
||||||
|
if (null != asyncCompactService) {
|
||||||
|
// Update the write client used by Async Compactor.
|
||||||
|
asyncCompactService.updateWriteClient(writeClient);
|
||||||
|
} else {
|
||||||
asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
|
asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
|
||||||
// Enqueue existing pending compactions first
|
// Enqueue existing pending compactions first
|
||||||
HoodieTableMetaClient meta =
|
HoodieTableMetaClient meta =
|
||||||
@@ -641,6 +645,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
throw new HoodieException(ie);
|
throw new HoodieException(ie);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -48,9 +48,6 @@ public class SchemaRegistryProvider extends SchemaProvider {
|
|||||||
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
|
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
|
||||||
}
|
}
|
||||||
|
|
||||||
private final Schema schema;
|
|
||||||
private final Schema targetSchema;
|
|
||||||
|
|
||||||
private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
|
private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
|
||||||
URL registry = new URL(registryUrl);
|
URL registry = new URL(registryUrl);
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
@@ -61,18 +58,6 @@ public class SchemaRegistryProvider extends SchemaProvider {
|
|||||||
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
|
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||||
super(props, jssc);
|
super(props, jssc);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
|
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 {
|
private static Schema getSchema(String registryUrl) throws IOException {
|
||||||
@@ -81,11 +66,22 @@ public class SchemaRegistryProvider extends SchemaProvider {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Schema getSourceSchema() {
|
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
|
@Override
|
||||||
public Schema getTargetSchema() {
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
Reference in New Issue
Block a user