Skip to content

Commit

Permalink
[Improve] Refactor Spark/Flink execution processor (#5595)
Browse files Browse the repository at this point in the history
  • Loading branch information
Hisoka-X authored Oct 16, 2023
1 parent b1d66c5 commit f083a86
Show file tree
Hide file tree
Showing 24 changed files with 959 additions and 355 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,25 @@ public static SeaTunnelDataType<SeaTunnelRow> convertToDataType(
}
}

public static List<CatalogTable> convertDataTypeToCatalogTables(
SeaTunnelDataType<?> seaTunnelDataType, String tableId) {
List<CatalogTable> catalogTables;
if (seaTunnelDataType instanceof MultipleRowType) {
catalogTables = new ArrayList<>();
for (String id : ((MultipleRowType) seaTunnelDataType).getTableIds()) {
catalogTables.add(
CatalogTableUtil.getCatalogTable(
id, ((MultipleRowType) seaTunnelDataType).getRowType(id)));
}
} else {
catalogTables =
Collections.singletonList(
CatalogTableUtil.getCatalogTable(
tableId, (SeaTunnelRowType) seaTunnelDataType));
}
return catalogTables;
}

public static CatalogTable buildWithConfig(ReadonlyConfig readonlyConfig) {
if (readonlyConfig.get(TableSchemaOptions.SCHEMA) == null) {
throw new RuntimeException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,7 @@
import org.apache.seatunnel.api.table.catalog.CatalogTableUtil;
import org.apache.seatunnel.api.table.catalog.TableIdentifier;
import org.apache.seatunnel.api.table.connector.TableSource;
import org.apache.seatunnel.api.table.type.MultipleRowType;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;

import org.slf4j.Logger;
Expand Down Expand Up @@ -62,7 +60,7 @@ public final class FactoryUtil {

private static final Logger LOG = LoggerFactory.getLogger(FactoryUtil.class);

static final String DEFAULT_ID = "default-identifier";
public static final String DEFAULT_ID = "default-identifier";

public static <T, SplitT extends SourceSplit, StateT extends Serializable>
List<Tuple2<SeaTunnelSource<T, SplitT, StateT>, List<CatalogTable>>>
Expand All @@ -86,19 +84,8 @@ public final class FactoryUtil {
SeaTunnelDataType<T> seaTunnelDataType = source.getProducedType();
final String tableId =
options.getOptional(CommonOptions.RESULT_TABLE_NAME).orElse(DEFAULT_ID);
if (seaTunnelDataType instanceof MultipleRowType) {
catalogTables = new ArrayList<>();
for (String id : ((MultipleRowType) seaTunnelDataType).getTableIds()) {
catalogTables.add(
CatalogTableUtil.getCatalogTable(
id, ((MultipleRowType) seaTunnelDataType).getRowType(id)));
}
} else {
catalogTables =
Collections.singletonList(
CatalogTableUtil.getCatalogTable(
tableId, (SeaTunnelRowType) seaTunnelDataType));
}
catalogTables =
CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, tableId);
}
LOG.info(
"get the CatalogTable from source {}: {}",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,176 @@
/*
* 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.seatunnel.core.starter.execution;

import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.common.CommonOptions;
import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
import org.apache.seatunnel.api.configuration.util.ConfigValidator;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.catalog.CatalogTableUtil;
import org.apache.seatunnel.api.table.connector.TableSource;
import org.apache.seatunnel.api.table.factory.Factory;
import org.apache.seatunnel.api.table.factory.FactoryException;
import org.apache.seatunnel.api.table.factory.TableSourceFactory;
import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext;
import org.apache.seatunnel.api.table.factory.TableTransformFactory;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.common.constants.JobMode;
import org.apache.seatunnel.common.utils.SeaTunnelException;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery;

import com.google.common.collect.Lists;

import java.net.URL;
import java.util.List;
import java.util.Optional;

import static org.apache.seatunnel.api.common.CommonOptions.PLUGIN_NAME;
import static org.apache.seatunnel.api.table.factory.FactoryUtil.DEFAULT_ID;
import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory;

/** The util used for Spark/Flink to create to SeaTunnelSource etc. */
public class PluginUtil {

protected static final String ENGINE_TYPE = "seatunnel";

public static SourceTableInfo createSource(
SeaTunnelFactoryDiscovery factoryDiscovery,
SeaTunnelSourcePluginDiscovery sourcePluginDiscovery,
PluginIdentifier pluginIdentifier,
Config pluginConfig,
JobContext jobContext) {
// get current thread classloader
ClassLoader classLoader =
Thread.currentThread()
.getContextClassLoader(); // try to find factory of this plugin

final ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(pluginConfig);
// try to find table source factory
final Optional<Factory> sourceFactory =
factoryDiscovery.createOptionalPluginInstance(pluginIdentifier);
final boolean fallback = isFallback(sourceFactory);
SeaTunnelSource source;
if (fallback) {
source = fallbackCreate(sourcePluginDiscovery, pluginIdentifier, pluginConfig);
} else {
// create source with source factory
TableSourceFactoryContext context =
new TableSourceFactoryContext(readonlyConfig, classLoader);
ConfigValidator.of(context.getOptions()).validate(sourceFactory.get().optionRule());
TableSource tableSource =
((TableSourceFactory) sourceFactory.get()).createSource(context);
source = tableSource.createSource();
}
source.setJobContext(jobContext);
ensureJobModeMatch(jobContext, source);
List<CatalogTable> catalogTables;
try {
catalogTables = source.getProducedCatalogTables();
} catch (UnsupportedOperationException e) {
// TODO remove it when all connector use `getProducedCatalogTables`
SeaTunnelDataType<?> seaTunnelDataType = source.getProducedType();
final String tableId =
readonlyConfig.getOptional(CommonOptions.RESULT_TABLE_NAME).orElse(DEFAULT_ID);
catalogTables =
CatalogTableUtil.convertDataTypeToCatalogTables(seaTunnelDataType, tableId);
}

if (catalogTables.size() != 1) {
throw new SeaTunnelException(
String.format("Unsupported table number: %d on flink", catalogTables.size()));
}
return new SourceTableInfo(source, catalogTables);
}

private static boolean isFallback(Optional<Factory> factory) {
if (!factory.isPresent()) {
return true;
}
try {
((TableSourceFactory) factory.get()).createSource(null);
} catch (Exception e) {
if (e instanceof UnsupportedOperationException
&& "The Factory has not been implemented and the deprecated Plugin will be used."
.equals(e.getMessage())) {
return true;
}
return true;
}
return false;
}

private static SeaTunnelSource fallbackCreate(
SeaTunnelSourcePluginDiscovery sourcePluginDiscovery,
PluginIdentifier pluginIdentifier,
Config pluginConfig) {
SeaTunnelSource source = sourcePluginDiscovery.createPluginInstance(pluginIdentifier);
source.prepare(pluginConfig);
return source;
}

public static TableTransformFactory createTransformFactory(
SeaTunnelTransformPluginDiscovery transformPluginDiscovery,
Config transformConfig,
List<URL> pluginJars) {
PluginIdentifier pluginIdentifier =
PluginIdentifier.of(
ENGINE_TYPE, "transform", transformConfig.getString(PLUGIN_NAME.key()));
final ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(transformConfig);
final String factoryId = readonlyConfig.get(PLUGIN_NAME);
ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
final TableTransformFactory factory =
discoverFactory(classLoader, TableTransformFactory.class, factoryId);
pluginJars.addAll(
transformPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier)));
return factory;
}

public static Optional<? extends Factory> createSinkFactory(
SeaTunnelFactoryDiscovery factoryDiscovery,
SeaTunnelSinkPluginDiscovery sinkPluginDiscovery,
Config sinkConfig,
List<URL> pluginJars) {
PluginIdentifier pluginIdentifier =
PluginIdentifier.of(ENGINE_TYPE, "sink", sinkConfig.getString(PLUGIN_NAME.key()));
pluginJars.addAll(
sinkPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier)));
try {
return factoryDiscovery.createOptionalPluginInstance(pluginIdentifier);
} catch (FactoryException e) {
return Optional.empty();
}
}

public static void ensureJobModeMatch(JobContext jobContext, SeaTunnelSource source) {
if (jobContext.getJobMode() == JobMode.BATCH
&& source.getBoundedness()
== org.apache.seatunnel.api.source.Boundedness.UNBOUNDED) {
throw new UnsupportedOperationException(
String.format(
"'%s' source don't support off-line job.", source.getPluginName()));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.seatunnel.core.starter.execution;

import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.catalog.CatalogTable;

import lombok.AllArgsConstructor;
import lombok.Data;

import java.util.List;

@Data
@AllArgsConstructor
public class SourceTableInfo {

private SeaTunnelSource source;

private List<CatalogTable> catalogTables;
}
Loading

0 comments on commit f083a86

Please sign in to comment.