Skip to content

Commit

Permalink
[Feature][seatunnel-translation-spark] Support Spark-3.3 - support ba…
Browse files Browse the repository at this point in the history
…tch read #2328
  • Loading branch information
zhaomin1423 committed Aug 1, 2022
1 parent c8e4abf commit 4252b5b
Show file tree
Hide file tree
Showing 16 changed files with 657 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,23 @@
import org.apache.seatunnel.shade.com.typesafe.config.Config;

import com.google.auto.service.AutoService;
import com.google.common.annotations.VisibleForTesting;

@AutoService(SeaTunnelSource.class)
public class FakeSource extends AbstractSingleSplitSource<SeaTunnelRow> {

private Config pluginConfig;
private SeaTunnelContext seaTunnelContext;

public FakeSource() {
}

@VisibleForTesting
public FakeSource(Config pluginConfig, SeaTunnelContext seaTunnelContext) {
this.pluginConfig = pluginConfig;
this.seaTunnelContext = seaTunnelContext;
}

@Override
public Boundedness getBoundedness() {
return JobMode.BATCH.equals(seaTunnelContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED;
Expand Down
3 changes: 2 additions & 1 deletion seatunnel-translation/seatunnel-translation-spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,8 @@
<artifactId>seatunnel-translation-spark</artifactId>
<packaging>pom</packaging>
<modules>
<module>seatunnel-translation-spark-2.4</module>
<module>seatunnel-translation-spark-common</module>
<module>seatunnel-translation-spark-2.4</module>
<module>seatunnel-translation-spark-3.3</module>
</modules>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.utils.SerializationUtils;
import org.apache.seatunnel.translation.spark.common.utils.Utils;
import org.apache.seatunnel.translation.spark.source.batch.BatchSourceReader;
import org.apache.seatunnel.translation.spark.source.continnous.ContinuousSourceReader;
import org.apache.seatunnel.translation.spark.source.micro.MicroBatchSourceReader;
Expand Down Expand Up @@ -87,7 +87,9 @@ public ContinuousReader createContinuousReader(Optional<StructType> rowTypeOptio
}

private SeaTunnelSource<SeaTunnelRow, ?, ?> getSeaTunnelSource(DataSourceOptions options) {
return SerializationUtils.stringToObject(options.get(Constants.SOURCE_SERIALIZATION)
.orElseThrow(() -> new UnsupportedOperationException("Serialization information for the SeaTunnelSource is required")));
String source = options.get(Constants.SOURCE_SERIALIZATION)
.orElseThrow(() ->
new UnsupportedOperationException("Serialization information for the SeaTunnelSource is required"));
return Utils.getSeaTunnelSource(source);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>seatunnel-translation-spark</artifactId>
<groupId>org.apache.seatunnel</groupId>
<version>${revision}</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>seatunnel-translation-spark-3.3</artifactId>

<properties>
<spark.version>3.3.0</spark.version>
<scala.binary.version>2.12</scala.binary.version>
<spark.scope>provided</spark.scope>
<scala.version>2.12.16</scala.version>
<commons-lang3.version>3.12.0</commons-lang3.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-translation-spark-common</artifactId>
<version>${project.version}</version>
</dependency>

<!--spark-->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>${spark.scope}</scope>
</dependency>

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>${spark.scope}</scope>
</dependency>

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>${spark.scope}</scope>
</dependency>

<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
<scope>${spark.scope}</scope>
</dependency>

<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-core-starter</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>connector-fake</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.translation.spark;

import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.translation.spark.common.utils.Utils;

import org.apache.spark.sql.connector.catalog.Table;
import org.apache.spark.sql.connector.catalog.TableProvider;
import org.apache.spark.sql.connector.expressions.Transform;
import org.apache.spark.sql.sources.DataSourceRegister;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;

import java.util.Map;

public class SeatunnelSource implements DataSourceRegister, TableProvider {

@Override
public String shortName() {
return "SeaTunnelSource-spark3.3";
}

@Override
public StructType inferSchema(CaseInsensitiveStringMap options) {
return null;
}

@Override
public Table getTable(StructType schema, Transform[] partitioning, Map<String, String> properties) {
SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(properties);
int parallelism = Integer.parseInt(
properties.getOrDefault(Constants.SOURCE_PARALLELISM, "1"));
return new SeatunnelTable(seaTunnelSource, parallelism);
}

private SeaTunnelSource<SeaTunnelRow, ?, ?> getSeaTunnelSource(Map<String, String> options) {
String source = options.get(Constants.SOURCE_SERIALIZATION);
return Utils.getSeaTunnelSource(source);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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.translation.spark;

import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.translation.spark.common.utils.TypeConverterUtils;
import org.apache.seatunnel.translation.spark.source.SeatunnelScanBuilder;

import com.google.common.collect.Sets;
import org.apache.spark.sql.connector.catalog.SupportsRead;
import org.apache.spark.sql.connector.catalog.Table;
import org.apache.spark.sql.connector.catalog.TableCapability;
import org.apache.spark.sql.connector.read.ScanBuilder;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;

import java.util.Set;

public class SeatunnelTable implements Table, SupportsRead {

private final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
private final Integer parallelism;

public SeatunnelTable(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism) {
this.source = source;
this.parallelism = parallelism;
}

@Override
public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
return new SeatunnelScanBuilder(source, parallelism);
}

@Override
public String name() {
return "SeaTunnel-Table";
}

@Override
public StructType schema() {
return (StructType) TypeConverterUtils.convert(source.getProducedType());
}

@Override
public Set<TableCapability> capabilities() {
return Sets.newHashSet(TableCapability.BATCH_READ);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.translation.spark.source;

import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SupportCoordinate;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;

import org.apache.spark.sql.connector.read.Batch;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReaderFactory;

public class SeatunnelBatch implements Batch {

private final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
private final Integer parallelism;

public SeatunnelBatch(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism) {
this.source = source;
this.parallelism = parallelism;
}

@Override
public InputPartition[] planInputPartitions() {
InputPartition[] partitions;
if (source instanceof SupportCoordinate) {
partitions = new SeatunnelInputPartition[1];
partitions[0] = new SeatunnelInputPartition(0);
} else {
partitions = new SeatunnelInputPartition[parallelism];
for (int subtaskId = 0; subtaskId < parallelism; subtaskId++) {
partitions[subtaskId] = new SeatunnelInputPartition(subtaskId);
}
}
return partitions;
}

@Override
public PartitionReaderFactory createReaderFactory() {
return new SeatunnelPartitionReaderFactory(source, parallelism);
}
}
Original file line number Diff line number Diff line change
@@ -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.seatunnel.translation.spark.source;

import org.apache.spark.sql.connector.read.InputPartition;

public class SeatunnelInputPartition implements InputPartition {

private final Integer subtaskId;

public SeatunnelInputPartition(Integer subtaskId) {
this.subtaskId = subtaskId;
}

public Integer getSubtaskId() {
return subtaskId;
}
}
Loading

0 comments on commit 4252b5b

Please sign in to comment.