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

V1.1.0 flink 1.9 pravega #62

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Set the start/end streamCut of stream while doing the read
  • Loading branch information
Wang,Chao committed Jun 8, 2020
commit a5661bc62f77715899bdcc86cdcfea0f38734f54
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,10 @@
import com.alibaba.alink.operator.common.io.csv.CsvUtil;
import com.alibaba.alink.operator.common.io.pravega.PravegaRowDeserializationSchema;
import com.alibaba.alink.params.io.PravegaSourceParams;
import io.pravega.client.ClientConfig;
import io.pravega.client.admin.StreamManager;
import io.pravega.client.stream.ReaderGroup;
import io.pravega.client.stream.StreamCut;
import io.pravega.connectors.flink.FlinkPravegaInputFormat;
import io.pravega.connectors.flink.PravegaConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand Down Expand Up @@ -37,6 +41,8 @@ protected Table initializeDataSource() {
String pravegaControllerUri = getPravegaControllerUri();
String pravegascope = getPravegaScope();
String pravegaStream = getPravegaStream();
StreamCut pravegaStartStreamCut = StreamCut.from(getPravegaStartStreamCut());
StreamCut pravegaEndStreamCut = StreamCut.from(getPravegaEndStreamCut());
String schemaStr = "event String";
//DeserializationSchema deserializationSchema = getPravegaDeserializer();
final String[] colNames = CsvUtil.getColNames(schemaStr);
Expand All @@ -58,13 +64,13 @@ protected Table initializeDataSource() {

FlinkPravegaInputFormat<Row> source = FlinkPravegaInputFormat.<Row>builder()
.withPravegaConfig(pravegaConfig)
.forStream(pravegaStream)
.forStream(pravegaStream, pravegaStartStreamCut, pravegaEndStreamCut)
.withDeserializationSchema(new PravegaRowDeserializationSchema(Row.class))
.build();

ExecutionEnvironment execEnv = MLEnvironmentFactory.get(getMLEnvironmentId()).getExecutionEnvironment();

DataSource<Row> data = execEnv.createInput(source, TypeInformation.of(Row.class)).name("PravegaBatch");
DataSource<Row> data = execEnv.createInput(source, TypeInformation.of(Row.class)).name("Pravega BatchReader");

return DataSetConversionUtil.toTable(getMLEnvironmentId(), data, colNames, colTypes);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,9 @@
import com.alibaba.alink.operator.common.io.csv.CsvUtil;
import com.alibaba.alink.operator.common.io.pravega.PravegaRowDeserializationSchema;
import com.alibaba.alink.params.io.PravegaSourceParams;
import io.pravega.client.ClientConfig;
import io.pravega.client.admin.StreamManager;
import io.pravega.client.stream.StreamCut;
import io.pravega.connectors.flink.FlinkPravegaReader;
import io.pravega.connectors.flink.PravegaConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand Down Expand Up @@ -40,6 +43,16 @@ protected Table initializeDataSource() {
String pravegaControllerUri = getPravegaControllerUri();
String pravegascope = getPravegaScope();
String pravegaStream = getPravegaStream();
StreamCut pravegaStartStreamCut = null;
if (getPravegaStartStreamCut().equals("UNBOUNDED"))
{
ClientConfig clientConfig= ClientConfig.builder().controllerURI(URI.create(pravegaControllerUri)).build();
StreamManager streamManager = StreamManager.create(clientConfig);
pravegaStartStreamCut = streamManager.getStreamInfo(pravegascope,pravegaStream).getTailStreamCut();
} else {
pravegaStartStreamCut = StreamCut.from(getPravegaStartStreamCut());
}
StreamCut pravegaEndStreamCut = StreamCut.from(getPravegaEndStreamCut());
String schemaStr = "event String";
final String[] colNames = CsvUtil.getColNames(schemaStr);
final TypeInformation[] colTypes = CsvUtil.getColTypes(schemaStr);
Expand All @@ -63,12 +76,12 @@ protected Table initializeDataSource() {

FlinkPravegaReader<Row> source = FlinkPravegaReader.<String>builder()
.withPravegaConfig(pravegaConfig)
.forStream(pravegaStream)
.forStream(pravegaStream, pravegaStartStreamCut, pravegaEndStreamCut)
.withDeserializationSchema(new PravegaRowDeserializationSchema(Row.class))
.build();

DataStream<Row> data = MLEnvironmentFactory.get(getMLEnvironmentId()).getStreamExecutionEnvironment()
.addSource(source).name("Pravega");
.addSource(source).name("Pravega StreamReader");
return DataStreamConversionUtil.toTable(getMLEnvironmentId(), data, colNames, colTypes);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,20 @@ public interface PravegaSourceParams<T> extends WithParams<T> {
.setRequired()
.setAlias(new String[]{"pravega.deserializer", "pravega_deserializer"})
.build();
ParamInfo<String> pravega_startStreamCut = ParamInfoFactory
.createParamInfo("pravega_startStreamCut ", String.class)
.setDescription("pravega startStreamCut")
.setHasDefaultValue("UNBOUNDED")
.setOptional()
.setAlias(new String[]{"pravega.startStreamCut", "pravega_startStreamCut"})
.build();
ParamInfo<String> pravega_endStreamCut = ParamInfoFactory
.createParamInfo("pravega_endtreamCut ", String.class)
.setDescription("pravega endStreamCut")
.setHasDefaultValue("UNBOUNDED")
.setOptional()
.setAlias(new String[]{"pravega.endStreamCut", "pravega_endtreamCut"})
.build();

default String getPravegaControllerUri() {
return get(PRAVEGA_CONTROLLER_URI);
Expand Down Expand Up @@ -64,4 +78,20 @@ default T setPravegaDeserializer(DeserializationSchema value) {
return set(pravega_deserializer, value);
}

default String getPravegaStartStreamCut() {
return get(pravega_startStreamCut);
}

default T setPravegaStartStreamCut(String value) {
return set(pravega_startStreamCut, value);
}

default String getPravegaEndStreamCut() {
return get(pravega_endStreamCut);
}

default T setPravegaEndStreamCut(String value) {
return set(pravega_endStreamCut, value);
}

}