[HUDI-3665] Support flink multiple versions (#5072)
This commit is contained in:
@@ -0,0 +1,27 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@code AbstractStreamOperator}.
|
||||
*/
|
||||
public abstract class AbstractStreamOperatorAdapter<O> extends AbstractStreamOperator<O> {
|
||||
}
|
||||
@@ -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.hudi.adapter;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@link AbstractStreamOperatorFactory}.
|
||||
*/
|
||||
public abstract class AbstractStreamOperatorFactoryAdapter<O>
|
||||
extends AbstractStreamOperatorFactory<O> implements YieldingOperatorFactory<O> {
|
||||
|
||||
public MailboxExecutorAdapter getMailboxExecutorAdapter() {
|
||||
return new MailboxExecutorAdapter(getMailboxExecutor());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,37 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.api.common.operators.MailboxExecutor;
|
||||
import org.apache.flink.util.function.ThrowingRunnable;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@link MailboxExecutor}.
|
||||
*/
|
||||
public class MailboxExecutorAdapter {
|
||||
private final MailboxExecutor executor;
|
||||
|
||||
public MailboxExecutorAdapter(MailboxExecutor executor) {
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
public void execute(ThrowingRunnable<? extends Exception> command, String description) {
|
||||
this.executor.execute(command, description);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter;
|
||||
|
||||
/**
|
||||
* Bridge class for shaded guava clazz {@code RateLimiter}.
|
||||
*/
|
||||
public class RateLimiterAdapter {
|
||||
private final RateLimiter rateLimiter;
|
||||
|
||||
private RateLimiterAdapter(double permitsPerSecond) {
|
||||
this.rateLimiter = RateLimiter.create(permitsPerSecond);
|
||||
}
|
||||
|
||||
public static RateLimiterAdapter create(double permitsPerSecond) {
|
||||
return new RateLimiterAdapter(permitsPerSecond);
|
||||
}
|
||||
|
||||
public void acquire() {
|
||||
this.rateLimiter.acquire();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.streaming.api.TimeCharacteristic;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.api.operators.StreamSourceContexts;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
|
||||
import org.apache.flink.streaming.runtime.tasks.StreamTask;
|
||||
|
||||
/**
|
||||
* Adapter utils.
|
||||
*/
|
||||
public class Utils {
|
||||
public static <O> SourceFunction.SourceContext<O> getSourceContext(
|
||||
TimeCharacteristic timeCharacteristic,
|
||||
ProcessingTimeService processingTimeService,
|
||||
StreamTask<?, ?> streamTask,
|
||||
Output<StreamRecord<O>> output,
|
||||
long watermarkInterval) {
|
||||
return StreamSourceContexts.getSourceContext(
|
||||
timeCharacteristic,
|
||||
processingTimeService,
|
||||
new Object(), // no actual locking needed
|
||||
output,
|
||||
watermarkInterval,
|
||||
-1,
|
||||
true);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@link Output}.
|
||||
*/
|
||||
public interface OutputAdapter<O> extends Output<O> {
|
||||
@Override
|
||||
default void emitWatermarkStatus(WatermarkStatus watermarkStatus) {
|
||||
// no operation
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.runtime.state.StateInitializationContext;
|
||||
|
||||
import java.util.OptionalLong;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@link StateInitializationContext}.
|
||||
*/
|
||||
public interface StateInitializationContextAdapter extends StateInitializationContext {
|
||||
@Override
|
||||
default OptionalLong getRestoredCheckpointId() {
|
||||
return OptionalLong.empty();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.api.common.accumulators.Accumulator;
|
||||
import org.apache.flink.metrics.groups.OperatorMetricGroup;
|
||||
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
|
||||
import org.apache.flink.runtime.execution.Environment;
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Adapter clazz for {@link StreamingRuntimeContext}.
|
||||
*/
|
||||
public class StreamingRuntimeContextAdapter extends StreamingRuntimeContext {
|
||||
|
||||
public StreamingRuntimeContextAdapter(AbstractStreamOperator<?> operator, Environment env,
|
||||
Map<String, Accumulator<?, ?>> accumulators) {
|
||||
super(operator, env, accumulators);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperatorMetricGroup getMetricGroup() {
|
||||
return UnregisteredMetricsGroup.createOperatorMetricGroup();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.adapter;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.TableEnvironment;
|
||||
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
|
||||
|
||||
/**
|
||||
* TableEnv for test goals.
|
||||
*/
|
||||
public class TestTableEnvs {
|
||||
|
||||
public static TableEnvironment getBatchTableEnv() {
|
||||
Configuration conf = new Configuration();
|
||||
// for batch upsert use cases: current suggestion is to disable these 2 options,
|
||||
// from 1.14, flink runtime execution mode has switched from streaming
|
||||
// to batch for batch execution mode(before that, both streaming and batch use streaming execution mode),
|
||||
// current batch execution mode has these limitations:
|
||||
//
|
||||
// 1. the keyed stream default to always sort the inputs by key;
|
||||
// 2. the batch state-backend requires the inputs sort by state key
|
||||
//
|
||||
// For our hudi batch pipeline upsert case, we rely on the consuming sequence for index records and data records,
|
||||
// the index records must be loaded first before data records for BucketAssignFunction to keep upsert semantics correct,
|
||||
// so we suggest disabling these 2 options to use streaming state-backend for batch execution mode
|
||||
// to keep the strategy before 1.14.
|
||||
conf.setBoolean("execution.sorted-inputs.enabled", false);
|
||||
conf.setBoolean("execution.batch-state-backend.enabled", false);
|
||||
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(conf);
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
return StreamTableEnvironment.create(execEnv, settings);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user