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

[api-draft][connector] Add simplified connector api #2041

Merged
merged 1 commit into from
Jun 27, 2022
Merged
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 @@ -24,12 +24,7 @@
*/
public interface SeaTunnelRuntimeEnvironment {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

xxxAware is a better class name when there is only onesetXXXmethod.
Refer to spring framewok org.springframework.web.context.ServletConfigAware ``org.springframework.context.ApplicationContextAware

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This's good idea! Can you contribute to it?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No problem


/**
* Returns the SeaTunnel runtime context.
*
* @return seaTunnelContext
*/
SeaTunnelContext getSeaTunnelContext();

void setSeaTunnelContext(SeaTunnelContext seaTunnelContext);
default void setSeaTunnelContext(SeaTunnelContext seaTunnelContext){
// nothing
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.common.constants.JobMode;

import java.io.Serializable;

Expand All @@ -42,10 +41,7 @@ public interface SeaTunnelSource<T, SplitT extends SourceSplit, StateT>
*
* @return the boundedness of this source.
*/
default Boundedness getBoundedness() {
return JobMode.BATCH.equals(getSeaTunnelContext().getJobMode()) ?
Boundedness.BOUNDED : Boundedness.UNBOUNDED;
}
Boundedness getBoundedness();

/**
* Get the data type of the records produced by this source.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
<artifactId>seatunnel-connectors-seatunnel</artifactId>

<modules>
<module>seatunnel-connector-seatunnel-common</module>
<module>seatunnel-connector-seatunnel-console</module>
<module>seatunnel-connector-seatunnel-fake</module>
<module>seatunnel-connector-seatunnel-kafka</module>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config.USERNAME;

import org.apache.seatunnel.api.common.PrepareFailException;
import org.apache.seatunnel.api.common.SeaTunnelContext;
import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.Boundedness;
Expand Down Expand Up @@ -58,7 +57,6 @@
@AutoService(SeaTunnelSource.class)
public class ClickhouseSource implements SeaTunnelSource<SeaTunnelRow, ClickhouseSourceSplit, ClickhouseSourceState> {

private SeaTunnelContext seaTunnelContext;
private List<ClickHouseNode> servers;
private SeaTunnelRowType rowTypeInfo;
private String sql;
Expand Down Expand Up @@ -139,13 +137,4 @@ public Serializer<ClickhouseSourceState> getEnumeratorStateSerializer() {
return new DefaultSerializer<>();
}

@Override
public SeaTunnelContext getSeaTunnelContext() {
return seaTunnelContext;
}

@Override
public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
this.seaTunnelContext = seaTunnelContext;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
<?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-connectors-seatunnel</artifactId>
<groupId>org.apache.seatunnel</groupId>
<version>${revision}</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>seatunnel-connector-seatunnel-common</artifactId>

<properties>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-api</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.connectors.seatunnel.common.sink;

import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
import org.apache.seatunnel.api.sink.SinkCommitter;
import org.apache.seatunnel.api.sink.SinkWriter;

import java.io.IOException;
import java.util.List;
import java.util.Optional;

public abstract class AbstractSimpleSink<T, StateT> implements SeaTunnelSink<T, StateT, Void, Void> {

@Override
public abstract AbstractSinkWriter<T, StateT> createWriter(SinkWriter.Context context) throws IOException;

@Override
public SinkWriter<T, Void, StateT> restoreWriter(SinkWriter.Context context, List<StateT> states) throws IOException {
return createWriter(context);
}

@Override
public final Optional<SinkCommitter<Void>> createCommitter() throws IOException {
return Optional.empty();
}

@Override
public final Optional<Serializer<Void>> getCommitInfoSerializer() {
return Optional.empty();
}

@Override
public final Optional<SinkAggregatedCommitter<Void, Void>> createAggregatedCommitter() throws IOException {
return Optional.empty();
}

@Override
public final Optional<Serializer<Void>> getAggregatedCommitInfoSerializer() {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,19 +15,21 @@
* limitations under the License.
*/

package org.apache.seatunnel.connectors.seatunnel.socket.source;
package org.apache.seatunnel.connectors.seatunnel.common.sink;

import org.apache.seatunnel.api.source.SourceSplit;
import org.apache.seatunnel.api.sink.SinkWriter;

public class SocketSourceSplit implements SourceSplit {
private final String splitId;
import java.util.Optional;

public SocketSourceSplit(String splitId) {
this.splitId = splitId;
public abstract class AbstractSinkWriter<T, StateT> implements SinkWriter<T, Void, StateT> {

@Override
public final Optional<Void> prepareCommit() {
return Optional.empty();
}

@Override
public String splitId() {
return splitId;
public final void abortPrepare() {
// nothing
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.connectors.seatunnel.common.source;

import org.apache.seatunnel.api.source.SourceEvent;
import org.apache.seatunnel.api.source.SourceReader;

import java.util.Collections;
import java.util.List;

public abstract class AbstractSingleSplitReader<T> implements SourceReader<T, SingleSplit> {

@Override
public final List<SingleSplit> snapshotState(long checkpointId) throws Exception {
return Collections.singletonList(new SingleSplit(snapshotStateToBytes(checkpointId)));
}

protected byte[] snapshotStateToBytes(long checkpointId) throws Exception {
// default nothing
return null;
}

@Override
public final void addSplits(List<SingleSplit> splits) {
if (splits.size() > 1) {
throw new UnsupportedOperationException("The single-split reader don't support reading multiple splits");
}
byte[] restoredState = splits.get(0).getState();
if (restoredState != null && restoredState.length > 0) {
restoreState(restoredState);
}
}

protected void restoreState(byte[] restoredState) {
// default nothing
}

@Override
public final void handleNoMoreSplits() {
// nothing
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
// default nothing
}

@Override
public final void handleSourceEvent(SourceEvent sourceEvent) {
// nothing
}
}
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.connectors.seatunnel.common.source;

import static com.google.common.base.Preconditions.checkArgument;

import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SourceReader;
import org.apache.seatunnel.api.source.SourceSplitEnumerator;

public abstract class AbstractSingleSplitSource<T> implements SeaTunnelSource<T, SingleSplit, SingleSplitEnumeratorState> {

@Override
public final AbstractSingleSplitReader<T> createReader(SourceReader.Context readerContext) throws Exception {
checkArgument(readerContext.getIndexOfSubtask() == 0, "Single split source allows only a single reader to be created.");
return createReader(new SingleSplitReaderContext(readerContext));
}

public abstract AbstractSingleSplitReader<T> createReader(SingleSplitReaderContext readerContext) throws Exception;

@Override
public final SourceSplitEnumerator<SingleSplit, SingleSplitEnumeratorState> createEnumerator(SourceSplitEnumerator.Context<SingleSplit> enumeratorContext) throws Exception {
return new SingleSplitEnumerator(enumeratorContext);
}

@Override
public final SourceSplitEnumerator<SingleSplit, SingleSplitEnumeratorState> restoreEnumerator(SourceSplitEnumerator.Context<SingleSplit> enumeratorContext, SingleSplitEnumeratorState checkpointState) throws Exception {
return createEnumerator(enumeratorContext);
}

@Override
public final Serializer<SingleSplitEnumeratorState> getEnumeratorStateSerializer() {
return new DefaultSerializer<>();
}

@Override
public final Serializer<SingleSplit> getSplitSerializer() {
return new DefaultSerializer<>();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -15,20 +15,23 @@
* limitations under the License.
*/

package org.apache.seatunnel.connectors.seatunnel.http.source;
package org.apache.seatunnel.connectors.seatunnel.common.source;

import org.apache.seatunnel.api.source.SourceSplit;

public class HttpSourceSplit implements SourceSplit {
public class SingleSplit implements SourceSplit {
private final byte[] state;

private final String splitId;
public SingleSplit(byte[] state) {
this.state = state;
}

public HttpSourceSplit(String splitId) {
this.splitId = splitId;
public byte[] getState() {
return state;
}

@Override
public String splitId() {
return this.splitId;
return "single";
}
}
Loading