Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[connector] support spark catalog #258

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
* limitations under the License.
*/

package com.alibaba.fluss.connector.flink.utils;
package com.alibaba.fluss.utils;

import com.alibaba.fluss.exception.DatabaseAlreadyExistException;
import com.alibaba.fluss.exception.DatabaseNotEmptyException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,12 @@
import com.alibaba.fluss.config.ConfigOptions;
import com.alibaba.fluss.config.Configuration;
import com.alibaba.fluss.connector.flink.lakehouse.LakeCatalog;
import com.alibaba.fluss.connector.flink.utils.CatalogExceptionUtils;
import com.alibaba.fluss.connector.flink.utils.FlinkConversions;
import com.alibaba.fluss.exception.FlussRuntimeException;
import com.alibaba.fluss.metadata.TableDescriptor;
import com.alibaba.fluss.metadata.TableInfo;
import com.alibaba.fluss.metadata.TablePath;
import com.alibaba.fluss.utils.CatalogExceptionUtils;
import com.alibaba.fluss.utils.ExceptionUtils;
import com.alibaba.fluss.utils.IOUtils;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright (c) 2024 Alibaba Group Holding Ltd.

Licensed 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">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-connector-spark</artifactId>
<version>0.6-SNAPSHOT</version>
</parent>

<artifactId>fluss-connector-spark-3.3</artifactId>

<name>Fluss : Connector : Spark : 3.3</name>

<properties>
<spark.version>3.3.3</spark.version>
</properties>

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

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

<dependency>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-connector-spark-common</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>shade-fluss</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<artifactSet>
<includes combine.children="append">
<include>com.alibaba.fluss:fluss-connector-spark-common</include>
<include>com.alibaba.fluss:fluss-client</include>
</includes>
</artifactSet>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright (c) 2024 Alibaba Group Holding Ltd.

Licensed 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">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-connector-spark</artifactId>
<version>0.6-SNAPSHOT</version>
</parent>

<artifactId>fluss-connector-spark-common</artifactId>

<name>Fluss : Connector : Spark : Common</name>

<properties>
<scala.common.version>2.12.15</scala.common.version>
<spark.common.version>3.5.3</spark.common.version>
</properties>

<dependencies>
<!-- Fluss dependency -->
<dependency>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-client</artifactId>
<version>${project.version}</version>
</dependency>

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

<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
<version>${scala.common.version}</version>
</dependency>

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

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${spark.common.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-slf4j2-impl</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-test</artifactId>
<version>${curator.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-server</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>com.alibaba.fluss</groupId>
<artifactId>fluss-test-utils</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,146 @@
/*
* Copyright (c) 2024 Alibaba Group Holding Ltd.
*
* Licensed 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 com.alibaba.fluss.connector.spark;

import com.alibaba.fluss.config.ConfigOption;
import com.alibaba.fluss.config.FlussConfigUtils;

import java.time.Duration;
import java.util.ArrayList;
import java.util.List;

import static com.alibaba.fluss.config.ConfigBuilder.key;

/** Options for spark connector. */
public class SparkConnectorOptions {

public static final ConfigOption<Integer> BUCKET_NUMBER =
key("bucket.num")
.intType()
.noDefaultValue()
.withDescription("The number of buckets of a Fluss table.");

public static final ConfigOption<String> BUCKET_KEY =
key("bucket.key")
.stringType()
.noDefaultValue()
.withDescription(
"Specific the distribution policy of the Fluss table. "
+ "Data will be distributed to each bucket according to the hash value of bucket-key. "
+ "If you specify multiple fields, delimiter is ','. "
+ "If the table is with primary key, you can't specific bucket key currently. "
+ "The bucket keys will always be the primary key. "
+ "If the table is not with primary key, you can specific bucket key, and when the bucket key is not specified, "
+ "the data will be distributed to each bucket randomly.");

public static final ConfigOption<String> BOOTSTRAP_SERVERS =
key("bootstrap.servers")
.stringType()
.noDefaultValue()
.withDescription(
"A list of host/port pairs to use for establishing the initial connection to the Fluss cluster. "
+ "The list should be in the form host1:port1,host2:port2,....");

public static final ConfigOption<String> PRIMARY_KEY =
key("primary.key")
.stringType()
.noDefaultValue()
.withDescription("the primary key of fluss table, such as key1,key2,...");

// --------------------------------------------------------------------------------------------
// Lookup specific options
// --------------------------------------------------------------------------------------------

public static final ConfigOption<Boolean> LOOKUP_ASYNC =
key("lookup.async")
.booleanType()
.defaultValue(true)
.withDescription("Whether to set async lookup. Default is true.");

// --------------------------------------------------------------------------------------------
// Scan specific options
// --------------------------------------------------------------------------------------------

public static final ConfigOption<ScanStartupMode> SCAN_STARTUP_MODE =
key("scan.startup.mode")
.enumType(ScanStartupMode.class)
.defaultValue(ScanStartupMode.INITIAL)
.withDescription(
"Optional startup mode for Fluss source. Default is 'initial'.");

public static final ConfigOption<String> SCAN_STARTUP_TIMESTAMP =
key("scan.startup.timestamp")
.stringType()
.noDefaultValue()
.withDescription(
"Optional timestamp for Fluss source in case of startup mode is timestamp. "
+ "The format is 'timestamp' or 'yyyy-MM-dd HH:mm:ss'. "
+ "Like '1678883047356' or '2023-12-09 23:09:12'.");

public static final ConfigOption<Duration> SCAN_PARTITION_DISCOVERY_INTERVAL =
key("scan.partition.discovery.interval")
.durationType()
.defaultValue(Duration.ofSeconds(10))
.withDescription(
"The interval in milliseconds for the Fluss source to discover "
+ "the new partitions for partitioned table while scanning."
+ " A non-positive value disables the partition discovery.");

// --------------------------------------------------------------------------------------------
// table storage specific options
// --------------------------------------------------------------------------------------------

public static final List<ConfigOption<?>> TABLE_OPTIONS =
new ArrayList<>(FlussConfigUtils.TABLE_OPTIONS.values());

// --------------------------------------------------------------------------------------------
// client specific options
// --------------------------------------------------------------------------------------------

public static final List<ConfigOption<?>> CLIENT_OPTIONS =
new ArrayList<>(FlussConfigUtils.CLIENT_OPTIONS.values());

// ------------------------------------------------------------------------------------------

/** Startup mode for the fluss scanner, see {@link #SCAN_STARTUP_MODE}. */
public enum ScanStartupMode {
INITIAL(
"initial",
"Performs an initial snapshot n the table upon first startup, "
+ "ans continue to read the latest changelog with exactly once guarantee. "
+ "If the table to read is a log table, the initial snapshot means "
+ "reading from earliest log offset. If the table to read is a primary key table, "
+ "the initial snapshot means reading a latest snapshot which "
+ "materializes all changes on the table."),
EARLIEST("earliest", "Start reading logs from the earliest offset."),
LATEST("latest", "Start reading logs from the latest offset."),
TIMESTAMP("timestamp", "Start reading logs from user-supplied timestamp.");

private final String value;
private final String description;

ScanStartupMode(String value, String description) {
this.value = value;
this.description = description;
}

@Override
public String toString() {
return value;
}
}
}
Loading