{
24 |
25 | @Override
26 | public Long deserialize(final DataPoint dataPoint) {
27 | return dataPoint.getField("longValue");
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/flink-connector-influxdb2/src/test/java/org/apache/flink/streaming/connectors/influxdb/util/InfluxDBContainer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package org.apache.flink.streaming.connectors.influxdb.util;
19 |
20 | import org.slf4j.Logger;
21 | import org.slf4j.LoggerFactory;
22 | import org.testcontainers.utility.DockerImageName;
23 |
24 | public final class InfluxDBContainer extends InfluxDBContainerCustom {
25 |
26 | private static final Logger LOG = LoggerFactory.getLogger(InfluxDBContainer.class);
27 |
28 | private static final DockerImageName DEFAULT_IMAGE_NAME =
29 | DockerImageName.parse("quay.io/influxdb/influxdb:v2.0.2");
30 |
31 | private InfluxDBContainer(final DockerImageName imageName) {
32 | super(imageName);
33 |
34 | }
35 |
36 | public static InfluxDBContainer createWithDefaultTag() {
37 | LOG.info("Starting influxDB test container with default tag {}", DEFAULT_IMAGE_NAME);
38 | return new InfluxDBContainer(DEFAULT_IMAGE_NAME);
39 | }
40 |
41 | }
42 |
--------------------------------------------------------------------------------
/flink-connector-influxdb2/src/test/resources/influx-setup.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | # Licensed to the Apache Software Foundation (ASF) under one
4 | # or more contributor license agreements. See the NOTICE file
5 | # distributed with this work for additional information
6 | # regarding copyright ownership. The ASF licenses this file
7 | # to you under the Apache License, Version 2.0 (the
8 | # "License"); you may not use this file except in compliance
9 | # with the License. You may obtain a copy of the License at
10 | #
11 | # http://www.apache.org/licenses/LICENSE-2.0
12 | #
13 | # Unless required by applicable law or agreed to in writing, software
14 | # distributed under the License is distributed on an "AS IS" BASIS,
15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 | # See the License for the specific language governing permissions and
17 | # limitations under the License.
18 |
19 | influx setup \
20 | --force \
21 | --username "${INFLUXDB_USER}" \
22 | --password "${INFLUXDB_PASSWORD}" \
23 | --bucket "${INFLUXDB_BUCKET}" \
24 | --org "${INFLUXDB_ORG}" \
25 | --retention "${INFLUXDB_RETENTION}""${INFLUXDB_RETENTION_UNIT}"
26 |
--------------------------------------------------------------------------------
/flink-connector-influxdb2/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | #
18 | # Set root logger level to OFF to not flood build logs
19 | # set manually to INFO for debugging purposes
20 | status=warn
21 | appender.console.type=Console
22 | appender.console.name=LogToConsole
23 | appender.console.layout.type=PatternLayout
24 | appender.console.layout.pattern=%d{HH:mm:ss.SSS} - %highlight{%5p} %style{%logger{36}}{cyan} - %m%n%throwable
25 | ### Logger test containers ###
26 | logger.testContainers.name=org.testcontainers
27 | logger.testContainers.level=WARN
28 | logger.testContainers.additivity=false
29 | logger.testContainers.appenderRef.console.ref=LogToConsole
30 | ### Logger Docker Java ###
31 | logger.dockerJava.name=com.github.dockerjava
32 | logger.dockerJava.level=WARN
33 | logger.dockerJava.additivity=false
34 | logger.dockerJava.appenderRef.console.ref=LogToConsole
35 | ### Logger Apache Flink ###
36 | logger.apacheFlink.name=org.apache.flink
37 | logger.apacheFlink.level=WARN
38 | logger.apacheFlink.additivity=false
39 | logger.apacheFlink.appenderRef.console.ref=LogToConsole
40 | ### Logger Apache Streaming Connectors ###
41 | logger.streamingConnectors.name=org.apache.flink.streaming.connectors
42 | logger.streamingConnectors.level=INFO
43 | logger.streamingConnectors.additivity=false
44 | logger.streamingConnectors.appenderRef.console.ref=LogToConsole
45 | # Root Logger
46 | rootLogger.level=OFF
47 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/ColumnSchemasFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.connectors.kudu.connector;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.kudu.ColumnSchema;
22 |
23 | import java.io.Serializable;
24 | import java.util.List;
25 |
26 | /**
27 | * Factory for creating {@link ColumnSchema}s to be used when creating a table that
28 | * does not currently exist in Kudu. Usable through {@link KuduTableInfo#createTableIfNotExists}.
29 | *
30 | * This factory implementation must be Serializable as it will be used directly in the Flink sources
31 | * and sinks.
32 | */
33 | @PublicEvolving
34 | public interface ColumnSchemasFactory extends Serializable {
35 |
36 | /**
37 | * Creates the columns of the Kudu table that will be used during the createTable operation.
38 | *
39 | * @return List of columns.
40 | */
41 | List getColumnSchemas();
42 |
43 | }
44 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/CreateTableOptionsFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.connectors.kudu.connector;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.kudu.client.CreateTableOptions;
22 |
23 | import java.io.Serializable;
24 |
25 | /**
26 | * Factory for creating {@link CreateTableOptions} to be used when creating a table that
27 | * does not currently exist in Kudu. Usable through {@link KuduTableInfo#createTableIfNotExists}.
28 | *
29 | * This factory implementation must be Serializable as it will be used directly in the Flink sources
30 | * and sinks.
31 | */
32 | @PublicEvolving
33 | public interface CreateTableOptionsFactory extends Serializable {
34 |
35 | /**
36 | * Creates the {@link CreateTableOptions} that will be used during the createTable operation.
37 | *
38 | * @return CreateTableOptions for creating the table.
39 | */
40 | CreateTableOptions getCreateTableOptions();
41 |
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/convertor/RowResultConvertor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.convertor;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.kudu.client.RowResult;
21 |
22 | import java.io.Serializable;
23 |
24 | /**
25 | * The RowResult object of Kudu is converted to correspond to the Flink internal row object
26 | *
27 | * @param Kudu rowResult Convertor format
28 | */
29 | @Internal
30 | public interface RowResultConvertor extends Serializable {
31 |
32 | /**
33 | * Convert Kudu RowResult to the corresponding format
34 | *
35 | * @param row Kudu RowResult Type
36 | * @return {@link T}
37 | */
38 | T convertor(RowResult row);
39 | }
40 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/convertor/RowResultRowConvertor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.convertor;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.types.Row;
21 | import org.apache.kudu.Schema;
22 | import org.apache.kudu.client.RowResult;
23 |
24 | /**
25 | * Transform the Kudu RowResult object into a Flink Row object
26 | */
27 | @Internal
28 | public class RowResultRowConvertor implements RowResultConvertor {
29 | @Override
30 | public Row convertor(RowResult row) {
31 | Schema schema = row.getColumnProjection();
32 |
33 | Row values = new Row(schema.getColumnCount());
34 | schema.getColumns().forEach(column -> {
35 | String name = column.getName();
36 | int pos = schema.getColumnIndex(name);
37 | if (row.isNull(name)) {
38 | return;
39 | }
40 | values.setField(pos, row.getObject(name));
41 | });
42 | return values;
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/failure/DefaultKuduFailureHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.failure;
18 |
19 | import org.apache.kudu.client.RowError;
20 |
21 | import java.io.IOException;
22 | import java.util.List;
23 | import java.util.stream.Collectors;
24 |
25 | /**
26 | * Default failure handling logic that doesn't do any handling but throws
27 | * an error.
28 | */
29 | public class DefaultKuduFailureHandler implements KuduFailureHandler {
30 |
31 | @Override
32 | public void onFailure(List failure) throws IOException {
33 | String errors = failure.stream()
34 | .map(error -> error.toString() + System.lineSeparator())
35 | .collect(Collectors.joining());
36 |
37 | throw new IOException("Error while sending value. \n " + errors);
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/failure/KuduFailureHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.failure;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.kudu.client.RowError;
21 |
22 | import java.io.IOException;
23 | import java.io.Serializable;
24 | import java.util.List;
25 |
26 | /**
27 | * Custom handling logic for errors resulting from trying to execute Kudu operations in the
28 | * {@link org.apache.flink.connectors.kudu.connector.writer.KuduWriter}
29 | */
30 | @PublicEvolving
31 | public interface KuduFailureHandler extends Serializable {
32 |
33 | /**
34 | * Handle a failed {@link List}.
35 | *
36 | * @param failure the cause of failure
37 | * @throws IOException if the sink should fail on this failure, the implementation should rethrow the throwable or a custom one
38 | */
39 | void onFailure(List failure) throws IOException;
40 |
41 | /**
42 | * Handle a ClassCastException. Default implementation rethrows the exception.
43 | *
44 | * @param e the cause of failure
45 | * @throws IOException if the casting failed
46 | */
47 | default void onTypeMismatch(ClassCastException e) throws IOException {
48 | throw new IOException("Class casting failed \n", e);
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/reader/KuduInputSplit.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.reader;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.core.io.LocatableInputSplit;
21 |
22 | @Internal
23 | public class KuduInputSplit extends LocatableInputSplit {
24 |
25 | private byte[] scanToken;
26 |
27 | /**
28 | * Creates a new KuduInputSplit
29 | *
30 | * @param splitNumber the number of the input split
31 | * @param hostnames The names of the hosts storing the data this input split refers to.
32 | */
33 | public KuduInputSplit(byte[] scanToken, final int splitNumber, final String[] hostnames) {
34 | super(splitNumber, hostnames);
35 |
36 | this.scanToken = scanToken;
37 | }
38 |
39 | public byte[] getScanToken() {
40 | return scanToken;
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduOperationMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.writer;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.kudu.client.KuduTable;
21 | import org.apache.kudu.client.Operation;
22 |
23 | import java.io.Serializable;
24 | import java.util.List;
25 |
26 | /**
27 | * Encapsulates the logic of mapping input records (of a DataStream) to operations
28 | * executed in Kudu. By allowing to return a list of operations we give flexibility
29 | * to the implementers to provide more sophisticated logic.
30 | *
31 | * @param Type of the input data
32 | */
33 | @PublicEvolving
34 | public interface KuduOperationMapper extends Serializable {
35 |
36 | /**
37 | * Create a list of operations to be executed by the {@link KuduWriter} for the
38 | * current input
39 | *
40 | * @param input input element
41 | * @param table table for which the operations should be created
42 | * @return List of operations to be executed on the table
43 | */
44 | List createOperations(T input, KuduTable table);
45 |
46 | }
47 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/RowOperationMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.writer;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.flink.types.Row;
21 |
22 | @PublicEvolving
23 | public class RowOperationMapper extends AbstractSingleOperationMapper {
24 |
25 | protected RowOperationMapper(String[] columnNames) {
26 | super(columnNames);
27 | }
28 |
29 | public RowOperationMapper(String[] columnNames, KuduOperation operation) {
30 | super(columnNames, operation);
31 | }
32 |
33 | @Override
34 | public Object getField(Row input, int i) {
35 | return input.getField(i);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/TupleOperationMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector.writer;
18 |
19 | import org.apache.flink.annotation.PublicEvolving;
20 | import org.apache.flink.api.java.tuple.Tuple;
21 |
22 | @PublicEvolving
23 | public class TupleOperationMapper extends AbstractSingleOperationMapper {
24 |
25 | protected TupleOperationMapper(String[] columnNames) {
26 | super(columnNames);
27 | }
28 |
29 | public TupleOperationMapper(String[] columnNames, KuduOperation operation) {
30 | super(columnNames, operation);
31 | }
32 |
33 | @Override
34 | public Object getField(T input, int i) {
35 | return input.getField(i);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/UpsertOperationMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.table;
18 |
19 | import org.apache.flink.annotation.Internal;
20 | import org.apache.flink.api.java.tuple.Tuple2;
21 | import org.apache.flink.connectors.kudu.connector.writer.AbstractSingleOperationMapper;
22 | import org.apache.flink.types.Row;
23 | import org.apache.kudu.client.KuduTable;
24 | import org.apache.kudu.client.Operation;
25 |
26 | import java.util.Optional;
27 |
28 | @Internal
29 | public class UpsertOperationMapper extends AbstractSingleOperationMapper> {
30 |
31 | public UpsertOperationMapper(String[] columnNames) {
32 | super(columnNames);
33 | }
34 |
35 | @Override
36 | public Object getField(Tuple2 input, int i) {
37 | return input.f1.getField(i);
38 | }
39 |
40 | @Override
41 | public Optional createBaseOperation(Tuple2 input, KuduTable table) {
42 | return Optional.of(input.f0 ? table.newUpsert() : table.newDelete());
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connectors.kudu.table.dynamic.KuduDynamicTableSourceSinkFactory
--------------------------------------------------------------------------------
/flink-connector-kudu/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.connectors.kudu.table.KuduTableFactory
17 | org.apache.flink.connectors.kudu.table.dynamic.catalog.KuduCatalogFactory
18 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/connector/KuduFilterInfoTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.connector;
18 |
19 | import org.apache.flink.table.data.binary.BinaryStringData;
20 |
21 | import org.apache.kudu.ColumnSchema;
22 | import org.apache.kudu.Type;
23 | import org.junit.jupiter.api.Test;
24 |
25 | import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
26 |
27 | public class KuduFilterInfoTest {
28 |
29 | @Test
30 | void testKuduFilterInfoWithBinaryStringData() {
31 | String filterValue = "someValue";
32 |
33 | KuduFilterInfo kuduFilterInfo = KuduFilterInfo.Builder.create("col")
34 | .equalTo(BinaryStringData.fromString(filterValue))
35 | .build();
36 |
37 | ColumnSchema colSchema = new ColumnSchema.ColumnSchemaBuilder("col", Type.STRING).build();
38 | assertDoesNotThrow(() -> kuduFilterInfo.toPredicate(colSchema));
39 | }
40 |
41 | }
42 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableTestUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.table;
18 |
19 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
20 | import org.apache.flink.table.api.EnvironmentSettings;
21 | import org.apache.flink.table.api.TableEnvironment;
22 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
23 |
24 | import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
25 |
26 | public class KuduTableTestUtils {
27 |
28 | public static StreamTableEnvironment createTableEnvWithBlinkPlannerStreamingMode(StreamExecutionEnvironment env) {
29 | EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build();
30 | StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
31 | tableEnv.getConfig().getConfiguration().setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
32 | return tableEnv;
33 | }
34 |
35 | public static TableEnvironment createTableEnvWithBlinkPlannerBatchMode() {
36 | EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
37 | TableEnvironment tableEnv = TableEnvironment.create(settings);
38 | tableEnv.getConfig().getConfiguration().setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
39 | return tableEnv;
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/writer/AbstractOperationTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.writer;
18 |
19 | import org.apache.flink.connectors.kudu.connector.KuduTestBase;
20 | import org.apache.kudu.Schema;
21 | import org.apache.kudu.client.*;
22 | import org.junit.jupiter.api.BeforeEach;
23 | import org.mockito.Mock;
24 | import org.mockito.MockitoAnnotations;
25 |
26 | import static org.mockito.Mockito.when;
27 |
28 | public abstract class AbstractOperationTest {
29 |
30 | public static final Schema tableSchema = KuduTestBase.booksTableInfo("test_table", true).getSchema();
31 | @Mock
32 | Insert mockInsert;
33 | @Mock
34 | Upsert mockUpsert;
35 | @Mock
36 | Update mockUpdate;
37 | @Mock
38 | Delete mockDelete;
39 | @Mock
40 | KuduTable mockTable;
41 | @Mock
42 | PartialRow mockPartialRow;
43 |
44 | @BeforeEach
45 | public void setup() {
46 | MockitoAnnotations.initMocks(this);
47 | when(mockInsert.getRow()).thenReturn(mockPartialRow);
48 | when(mockUpsert.getRow()).thenReturn(mockPartialRow);
49 | when(mockUpdate.getRow()).thenReturn(mockPartialRow);
50 | when(mockDelete.getRow()).thenReturn(mockPartialRow);
51 | when(mockTable.newInsert()).thenReturn(mockInsert);
52 | when(mockTable.newUpsert()).thenReturn(mockUpsert);
53 | when(mockTable.newUpdate()).thenReturn(mockUpdate);
54 | when(mockTable.newDelete()).thenReturn(mockDelete);
55 | when(mockTable.getSchema()).thenReturn(tableSchema);
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/writer/RowDataUpsertOperationMapperTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.connectors.kudu.writer;
18 |
19 | import org.apache.flink.connectors.kudu.connector.KuduTestBase;
20 | import org.apache.flink.connectors.kudu.connector.writer.RowDataUpsertOperationMapper;
21 | import org.apache.flink.table.data.RowData;
22 | import org.apache.kudu.client.Operation;
23 | import org.junit.jupiter.api.Assertions;
24 | import org.junit.jupiter.api.Test;
25 |
26 | import java.util.List;
27 |
28 | import static org.junit.Assert.assertEquals;
29 | import static org.mockito.Mockito.verify;
30 |
31 | /**
32 | * Unit Tests for {@link RowDataUpsertOperationMapper}.
33 | */
34 | public class RowDataUpsertOperationMapperTest extends AbstractOperationTest {
35 |
36 | @Test
37 | void testGetField() {
38 | RowDataUpsertOperationMapper mapper =
39 | new RowDataUpsertOperationMapper(KuduTestBase.booksTableSchema());
40 | RowData inputRow = KuduTestBase.booksRowData().get(0);
41 |
42 | Assertions.assertEquals(inputRow.getInt(0), mapper.getField(inputRow, 0));
43 | }
44 |
45 |
46 | @Test
47 | void testCorrectOperationUpsert() {
48 | RowDataUpsertOperationMapper mapper =
49 | new RowDataUpsertOperationMapper(KuduTestBase.booksTableSchema());
50 | RowData inputRow = KuduTestBase.booksRowData().get(0);
51 |
52 | List operations = mapper.createOperations(inputRow, mockTable);
53 |
54 | assertEquals(1, operations.size());
55 | verify(mockTable).newUpsert();
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # This file ensures that tests executed from the IDE show log output
20 |
21 | log4j.rootLogger=WARN, console
22 |
23 | # Log all infos in the given file
24 | log4j.appender.console=org.apache.log4j.ConsoleAppender
25 | log4j.appender.console.target=System.err
26 | log4j.appender.console.layout=org.apache.log4j.PatternLayout
27 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
28 |
--------------------------------------------------------------------------------
/flink-connector-kudu/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | rootLogger.level = INFO
22 | rootLogger.appenderRef.test.ref = TestLogger
23 |
24 | appender.testlogger.name = TestLogger
25 | appender.testlogger.type = CONSOLE
26 | appender.testlogger.target = SYSTEM_ERR
27 | appender.testlogger.layout.type = PatternLayout
28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
29 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/main/scala/org/apache/flink/streaming/connectors/netty/example/HttpReceiverSource.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction
20 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
21 |
22 | /**
23 | * Http receiver source is used for receiving pushed http request.
24 | * It work for two step:
25 | * 1. start netty server with an un-used port when Flink get start
26 | * 2. after started netty, call back [[callbackUrl]] for register current connector to
27 | * message service, user can push http message to this address.
28 | * {{{
29 | * // for example:
30 | * val env = StreamExecutionEnvironment.getExecutionEnvironment
31 | * env.addSource(new TcpReceiverSource(7070, Some("http://localhost:9090/cb")))
32 | * }}}
33 | *
34 | * @param paramKey the http query param key
35 | * @param tryPort try to use this point, if this point is used then try a new port
36 | * @param callbackUrl register connector's ip and port to a third service
37 | */
38 | final class HttpReceiverSource(
39 | paramKey: String,
40 | tryPort: Int,
41 | callbackUrl: Option[String] = None
42 | ) extends RichParallelSourceFunction[String] {
43 | private var server: HttpServer = _
44 |
45 | override def cancel(): Unit = server.close()
46 |
47 | override def run(ctx: SourceContext[String]): Unit = {
48 | server = new HttpServer(ctx, paramKey)
49 | server.start(tryPort, callbackUrl)
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/main/scala/org/apache/flink/streaming/connectors/netty/example/LineParserTrait.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import scala.language.higherKinds
20 |
21 | /**
22 | * parse data line
23 | */
24 | trait PayloadParse[S, T] {
25 | def parse(line: S): T
26 | }
27 |
28 | trait StringParser[T] extends PayloadParse[String, T]
29 |
30 | trait BytesParser[Array[Byte], T] extends PayloadParse[Array[Byte], T]
31 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/main/scala/org/apache/flink/streaming/connectors/netty/example/ServerTrait.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import java.io.Closeable
20 | import java.net.{InetSocketAddress, URLEncoder}
21 |
22 | /**
23 | * Server trait for define server behave.
24 | * Port and callback url is required.
25 | */
26 | trait ServerTrait extends Closeable {
27 |
28 | def start(tryPort: Int, callbackUrl: Option[String]): InetSocketAddress = {
29 | NettyUtil.startServiceOnPort(tryPort, (p: Int) => startNettyServer(p, callbackUrl))
30 | }
31 |
32 | def startNettyServer(portNotInUse: Int, callbackUrl: Option[String]): InetSocketAddress
33 |
34 | def register(address: InetSocketAddress, callbackUrl: Option[String]): Unit = {
35 | callbackUrl match {
36 | case Some(url) =>
37 | val ip = address.getAddress.getHostAddress
38 | val newIp = if (ip.startsWith("0") || ip.startsWith("127")) {
39 | NettyUtil.findLocalInetAddress().getHostAddress
40 | } else {
41 | ip
42 | }
43 | val port = address.getPort
44 | val param = s"ip=${URLEncoder.encode(newIp, "UTF-8")}&port=$port"
45 | val callUrl = if (url.endsWith("?")) param else "?" + param
46 | NettyUtil.sendGetRequest(url + callUrl)
47 | case _ =>
48 | }
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/main/scala/org/apache/flink/streaming/connectors/netty/example/TcpHandler.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.netty.example
19 |
20 | import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler}
21 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
22 | import org.slf4j.LoggerFactory
23 |
24 | /**
25 | * process netty stream data, add to flink
26 | */
27 | private class TcpHandler(sctx: SourceContext[String]) extends SimpleChannelInboundHandler[String] {
28 | private lazy val logger = LoggerFactory.getLogger(getClass)
29 |
30 | override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = {
31 | sctx.collect(msg)
32 | }
33 |
34 | override def channelActive(ctx: ChannelHandlerContext): Unit = {
35 | logger.info(s"tcp channel active, remote address:${ctx.channel().remoteAddress()}")
36 | }
37 |
38 | override def channelReadComplete(ctx: ChannelHandlerContext): Unit = ctx.flush
39 |
40 | override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable) {
41 | logger.error(s"netty server channel ${ctx.channel()} error", cause)
42 | ctx.close()
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/main/scala/org/apache/flink/streaming/connectors/netty/example/TcpReceiverSource.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.netty.example
19 |
20 | import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction
21 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
22 |
23 | /**
24 | * A end-to-end source, build a keep-alive tcp channel by netty.
25 | *
26 | * When this source stream get start, listen a provided tcp port, receive stream data sent from
27 | * the place where origin data generated.
28 | * {{{
29 | * // for example:
30 | * val env = StreamExecutionEnvironment.getExecutionEnvironment
31 | * env.addSource(new TcpReceiverSource("msg", 7070, Some("http://localhost:9090/cb")))
32 | * }}}
33 | * The features provide by this source:
34 | * 1. source run as a netty tcp server
35 | * 2. listen provided tcp port, if the port is in used,
36 | * increase the port number between 1024 to 65535
37 | * 3. callback the provided url to report the real port to listen
38 | *
39 | * @param tryPort the tcp port to start, if port Collision, retry a new port
40 | * @param callbackUrl when netty server started, report the ip and port to this url
41 | */
42 | final class TcpReceiverSource(
43 | tryPort: Int,
44 | callbackUrl: Option[String] = None
45 | ) extends RichParallelSourceFunction[String] {
46 | private var server: TcpServer = _
47 |
48 | override def cancel(): Unit = server.close()
49 |
50 | override def run(ctx: SourceContext[String]): Unit = {
51 | server = TcpServer(tryPort, ctx)
52 | server.start(tryPort, callbackUrl)
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/test/resources/log4j-test.properties:
--------------------------------------------------------------------------------
1 | ###############################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | log4j.rootLogger=OFF, console
20 |
21 | # -----------------------------------------------------------------------------
22 | # Console (use 'console')
23 | # -----------------------------------------------------------------------------
24 | log4j.appender.console=org.apache.log4j.ConsoleAppender
25 | log4j.appender.console.layout=org.apache.flink.util.MavenForkNumberPrefixLayout
26 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
27 |
28 | # -----------------------------------------------------------------------------
29 | # File (use 'file')
30 | # -----------------------------------------------------------------------------
31 | log4j.appender.file=org.apache.log4j.FileAppender
32 | log4j.appender.file.file=${log.dir}/{$mvn.forkNumber}.log
33 | log4j.appender.file.append=false
34 | log4j.appender.file.layout=org.apache.log4j.PatternLayout
35 | log4j.appender.file.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
36 |
37 | # suppress the irrelevant (wrong) warnings from the netty channel handler
38 | log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console
39 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
18 |
19 |
20 |
21 |
22 | %d{HH:mm:ss.SSS} [%thread] [%X{sourceThread} - %X{akkaSource}] %-5level %logger{60} - %msg%n
23 |
24 |
25 |
26 |
27 |
28 |
29 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/HttpSourceExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import java.net.URLEncoder
20 | import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
21 |
22 | import com.alibaba.fastjson.JSONObject
23 |
24 | import scala.util.Random
25 |
26 | /**
27 | * http client
28 | */
29 | object HttpSourceExample extends BaseTest {
30 |
31 | def main(args: Array[String]): Unit = {
32 | val queue = new LinkedBlockingQueue[JSONObject]()
33 |
34 | run(() => new MonitorServer(queue).start(9090))
35 | run(() => StreamSqlExample.main(Array("--http", "true")))
36 |
37 | Thread.sleep(5000)
38 |
39 | while (true) {
40 | val json = queue.poll(Int.MaxValue, TimeUnit.SECONDS)
41 | logger.info("====request register from netty tcp source: " + json)
42 | val url = s"http://${json.getString("ip")}:${json.getString("port")}/payload?msg="
43 | schedule(5, () => {
44 | val line = s"${Random.nextInt(5)},abc,${Random.nextInt(100)}"
45 | sendGetRequest(url + URLEncoder.encode(line, "UTF-8"))
46 | })
47 | }
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/NettyClientHandler.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import io.netty.channel._
20 | import org.slf4j.LoggerFactory
21 |
22 | import scala.util.Random
23 |
24 |
25 | final class NettyClientHandler extends SimpleChannelInboundHandler[String] {
26 | private lazy val logger = LoggerFactory.getLogger(getClass)
27 |
28 | override def channelActive(ctx: ChannelHandlerContext): Unit = {
29 | val ch = ctx.channel()
30 | logger.info(s"active channel: $ch")
31 | }
32 |
33 | override def channelInactive(ctx: ChannelHandlerContext) {
34 | val ch = ctx.channel()
35 | logger.info(s"inactive channel: $ch")
36 | }
37 |
38 | override def channelRead0(ctx: ChannelHandlerContext, msg: String) {
39 | logger.info("receive message:" + msg)
40 | ctx.writeAndFlush(Random.nextLong() + ",sjk," + Random.nextInt())
41 | }
42 |
43 | override def channelReadComplete(ctx: ChannelHandlerContext): Unit = ctx.flush
44 |
45 | override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable) {
46 | cause.printStackTrace()
47 | ctx.close
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/TcpSourceExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.netty.example
18 |
19 | import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
20 |
21 | import com.alibaba.fastjson.JSONObject
22 |
23 | import scala.util.Random
24 |
25 | object TcpSourceExample extends BaseTest {
26 |
27 | def main(args: Array[String]): Unit = {
28 | val queue = new LinkedBlockingQueue[JSONObject]()
29 |
30 | // 1. register server, wait for flink netty source server started
31 | run(() => new MonitorServer(queue).start(9090))
32 | // 2. start flink job
33 | run(() => StreamSqlExample.main(Array("--tcp", "true")))
34 |
35 | Thread.sleep(5000)
36 |
37 | // 3. sending message to netty source continuously
38 | while (true) {
39 | logger.info("==============")
40 | val json = queue.poll(Int.MaxValue, TimeUnit.SECONDS)
41 | logger.info("====request register from netty tcp source: " + json)
42 | val client = new NettyClient("localhost", json.getInteger("port"))
43 | client.run()
44 | schedule(5, () => {
45 | val line = Random.nextLong() + ",sjk," + Random.nextInt()
46 | client.send(line)
47 | })
48 | }
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-pinot/docs/images/PinotSinkGlobalCommitter_combine.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/apache/bahir-flink/4a8f60f03f7caeb5421c7c88fa34a3ba419b61ee/flink-connector-pinot/docs/images/PinotSinkGlobalCommitter_combine.png
--------------------------------------------------------------------------------
/flink-connector-pinot/docs/images/PinotSinkWriter.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/apache/bahir-flink/4a8f60f03f7caeb5421c7c88fa34a3ba419b61ee/flink-connector-pinot/docs/images/PinotSinkWriter.png
--------------------------------------------------------------------------------
/flink-connector-pinot/docs/images/PinotSinkWriter_prepareCommit.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/apache/bahir-flink/4a8f60f03f7caeb5421c7c88fa34a3ba419b61ee/flink-connector-pinot/docs/images/PinotSinkWriter_prepareCommit.png
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommittable.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.committer;
20 |
21 | import org.apache.flink.annotation.Internal;
22 |
23 | import java.io.Serializable;
24 | import java.util.List;
25 |
26 | import static org.apache.flink.util.Preconditions.checkNotNull;
27 |
28 | /**
29 | * Global committable references all data files that will be committed during checkpointing.
30 | */
31 | @Internal
32 | public class PinotSinkGlobalCommittable implements Serializable {
33 | private final List dataFilePaths;
34 | private final long minTimestamp;
35 | private final long maxTimestamp;
36 |
37 | /**
38 | * @param dataFilePaths List of paths to data files on shared file system
39 | * @param minTimestamp Minimum timestamp of all objects in all data files
40 | * @param maxTimestamp Maximum timestamp of all objects in all data files
41 | */
42 | public PinotSinkGlobalCommittable(List dataFilePaths, long minTimestamp, long maxTimestamp) {
43 | this.dataFilePaths = checkNotNull(dataFilePaths);
44 | this.minTimestamp = minTimestamp;
45 | this.maxTimestamp = maxTimestamp;
46 | }
47 |
48 | public List getDataFilePaths() {
49 | return dataFilePaths;
50 | }
51 |
52 | public long getMinTimestamp() {
53 | return minTimestamp;
54 | }
55 |
56 | public long getMaxTimestamp() {
57 | return maxTimestamp;
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/exceptions/PinotControllerApiException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.exceptions;
20 |
21 | import org.apache.flink.annotation.Internal;
22 |
23 | import java.io.IOException;
24 |
25 | /**
26 | * Pinot controller API exception wrapper
27 | */
28 | @Internal
29 | public class PinotControllerApiException extends IOException {
30 |
31 | public PinotControllerApiException(String reason) {
32 | super(reason);
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.external;
20 |
21 | import org.apache.flink.api.connector.sink.SinkWriter;
22 |
23 | import java.io.Serializable;
24 | import java.util.concurrent.TimeUnit;
25 |
26 | /**
27 | * Defines the interface for event time extractors
28 | *
29 | * @param Type of incoming elements
30 | */
31 | public interface EventTimeExtractor extends Serializable {
32 |
33 | /**
34 | * Extracts event time from incoming elements.
35 | *
36 | * @param element Incoming element
37 | * @param context Context of SinkWriter
38 | * @return timestamp
39 | */
40 | long getEventTime(IN element, SinkWriter.Context context);
41 |
42 | /**
43 | * @return Name of column in Pinot target table that contains the timestamp.
44 | */
45 | String getTimeColumn();
46 |
47 | /**
48 | * @return Unit of the time column in the Pinot target table.
49 | */
50 | TimeUnit getSegmentTimeUnit();
51 | }
52 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.external;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * Defines the interface for serializing incoming elements to JSON format.
25 | * The JSON format is expected during Pinot segment creation.
26 | *
27 | * @param Type of incoming elements
28 | */
29 | public abstract class JsonSerializer implements Serializable {
30 |
31 | public abstract String toJson(IN element);
32 | }
33 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.pinot.filesystem;
19 |
20 | import java.io.IOException;
21 | import java.io.Serializable;
22 | import java.util.List;
23 |
24 | /**
25 | * Defines the interaction with a shared filesystem. The shared filesystem must be accessible from all
26 | * nodes within the cluster than run a partition of the {@link org.apache.flink.streaming.connectors.pinot.PinotSink}.
27 | */
28 | public interface FileSystemAdapter extends Serializable {
29 |
30 | /**
31 | * Writes a list of serialized elements to the shared filesystem.
32 | *
33 | * @param elements List of serialized elements
34 | * @return Path identifying the remote file
35 | * @throws IOException
36 | */
37 | String writeToSharedFileSystem(List elements) throws IOException;
38 |
39 | /**
40 | * Reads a previously written list of serialized elements from the shared filesystem.
41 | *
42 | * @param path Path returned by {@link #writeToSharedFileSystem}
43 | * @return List of serialized elements read from the shared filesystem
44 | * @throws IOException
45 | */
46 | List readFromSharedFileSystem(String path) throws IOException;
47 |
48 | /**
49 | * Deletes a file from the shared filesystem
50 | *
51 | * @param path Path returned by {@link #writeToSharedFileSystem}
52 | * @throws IOException
53 | */
54 | void deleteFromSharedFileSystem(String path) throws IOException;
55 | }
56 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.pinot.segment.name;
19 |
20 | import org.apache.pinot.core.segment.name.SegmentNameGenerator;
21 |
22 | import java.io.Serializable;
23 |
24 | /**
25 | * Defines the segment name generator interface that is used to generate segment names. The segment
26 | * name generator is required to be serializable. We expect users to inherit from
27 | * {@link PinotSinkSegmentNameGenerator} in case they want to define their custom name generator.
28 | */
29 | public interface PinotSinkSegmentNameGenerator extends SegmentNameGenerator, Serializable {
30 | }
31 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/v2/committer/PinotSinkCommittable.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.v2.committer;
20 |
21 | import org.apache.flink.annotation.Internal;
22 |
23 | import java.io.Serializable;
24 |
25 | import static org.apache.flink.util.Preconditions.checkNotNull;
26 |
27 | /**
28 | * The PinotSinkCommittable is required for sharing committables with the
29 | * {@link PinotSinkCommittable} instance
30 | */
31 | @Internal
32 | public class PinotSinkCommittable implements Serializable {
33 | private final String dataFilePath;
34 | private final long minTimestamp;
35 | private final long maxTimestamp;
36 |
37 | /**
38 | * @param dataFilePath Path referencing a file on the shared filesystem defined via {@link org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter}
39 | * @param minTimestamp The minimum timestamp of all the elements contained in {@link #dataFilePath}
40 | * @param maxTimestamp The maximum timestamp of all the elements contained in {@link #dataFilePath}
41 | */
42 | public PinotSinkCommittable(String dataFilePath, long minTimestamp, long maxTimestamp) {
43 | this.dataFilePath = checkNotNull(dataFilePath);
44 | this.minTimestamp = minTimestamp;
45 | this.maxTimestamp = maxTimestamp;
46 | }
47 |
48 | public String getDataFilePath() {
49 | return dataFilePath;
50 | }
51 |
52 | public long getMinTimestamp() {
53 | return minTimestamp;
54 | }
55 |
56 | public long getMaxTimestamp() {
57 | return maxTimestamp;
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/v2/external/EventTimeExtractor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.v2.external;
20 |
21 | import org.apache.flink.api.connector.sink2.SinkWriter;
22 |
23 | import java.io.Serializable;
24 | import java.util.concurrent.TimeUnit;
25 |
26 | /**
27 | * Defines the interface for event time extractors
28 | *
29 | * @param Type of incoming elements
30 | */
31 | public interface EventTimeExtractor extends Serializable {
32 |
33 | /**
34 | * Extracts event time from incoming elements.
35 | *
36 | * @param element Incoming element
37 | * @param context Context of SinkWriter
38 | * @return timestamp
39 | */
40 | long getEventTime(IN element, SinkWriter.Context context);
41 |
42 | /**
43 | * @return Name of column in Pinot target table that contains the timestamp.
44 | */
45 | String getTimeColumn();
46 |
47 | /**
48 | * @return Unit of the time column in the Pinot target table.
49 | */
50 | TimeUnit getSegmentTimeUnit();
51 | }
52 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/v2/external/JsonSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.v2.external;
20 |
21 | import java.io.Serializable;
22 |
23 | /**
24 | * Defines the interface for serializing incoming elements to JSON format.
25 | * The JSON format is expected during Pinot segment creation.
26 | *
27 | * @param Type of incoming elements
28 | */
29 | public abstract class JsonSerializer implements Serializable {
30 |
31 | public abstract String toJson(IN element);
32 | }
33 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/v2/writer/PinotWriterState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | package org.apache.flink.streaming.connectors.pinot.v2.writer;
20 |
21 | import java.io.Serializable;
22 | import java.util.List;
23 |
24 | public class PinotWriterState implements Serializable {
25 |
26 | private final List serializedElements;
27 | private final long minTimestamp;
28 | private final long maxTimestamp;
29 |
30 | public PinotWriterState(List serializedElements, long minTimestamp, long maxTimestamp) {
31 | this.serializedElements = serializedElements;
32 | this.minTimestamp = minTimestamp;
33 | this.maxTimestamp = maxTimestamp;
34 | }
35 |
36 | public List getSerializedElements() {
37 | return serializedElements;
38 | }
39 |
40 | public long getMinTimestamp() {
41 | return minTimestamp;
42 | }
43 |
44 | public long getMaxTimestamp() {
45 | return maxTimestamp;
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/test/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # This file ensures that tests executed from the IDE show log output
20 |
21 | log4j.rootLogger=INFO, console
22 |
23 | # Log all infos in the given file
24 | log4j.appender.console=org.apache.log4j.ConsoleAppender
25 | log4j.appender.console.target=System.out
26 | log4j.appender.console.layout=org.apache.log4j.PatternLayout
27 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
28 |
29 |
30 | status=warn
31 | appender.console.type=Console
32 | appender.console.name=LogToConsole
33 | appender.console.layout.type=PatternLayout
34 | appender.console.layout.pattern=%d{HH:mm:ss.SSS} - %highlight{%5p} %style{%logger{36}}{cyan} - %m%n%throwable
35 |
36 | ### Logger Apache Flink ###
37 | logger.apacheFlink.name=org.apache.flink.streaming.connectors.pinot
38 | logger.apacheFlink.level=debug
39 | logger.apacheFlink.additivity=false
40 | logger.apacheFlink.appenderRef.console.ref=LogToConsole
41 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/test/resources/log4j2-test.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | #
18 | # Set root logger level to OFF to not flood build logs
19 | # set manually to INFO for debugging purposes
20 | status=WARN
21 | appender.console.type=Console
22 | appender.console.name=LogToConsole
23 | appender.console.layout.type=PatternLayout
24 | appender.console.layout.pattern=%d{HH:mm:ss.SSS} - %highlight{%5p} %style{%logger{36}}{cyan} - %m%n%throwable
25 | ### Logger test containers ###
26 | logger.testContainers.name=org.testcontainers
27 | logger.testContainers.level=WARN
28 | logger.testContainers.additivity=false
29 | logger.testContainers.appenderRef.console.ref=LogToConsole
30 | ### Logger Docker Java ###
31 | logger.dockerJava.name=com.github.dockerjava
32 | logger.dockerJava.level=WARN
33 | logger.dockerJava.additivity=false
34 | logger.dockerJava.appenderRef.console.ref=LogToConsole
35 | ### Logger Apache Flink ###
36 | logger.apacheFlink.name=org.apache.flink
37 | logger.apacheFlink.level=WARN
38 | logger.apacheFlink.additivity=false
39 | logger.apacheFlink.appenderRef.console.ref=LogToConsole
40 | ### Logger Apache Streaming Connectors ###
41 | logger.streamingConnectors.name=org.apache.flink.streaming.connectors
42 | logger.streamingConnectors.level=WARN
43 | logger.streamingConnectors.additivity=false
44 | logger.streamingConnectors.appenderRef.console.ref=LogToConsole
45 | # Root Logger
46 | rootLogger.level=OFF
47 |
--------------------------------------------------------------------------------
/flink-connector-pinot/src/test/resources/pinot-broker/pinot-broker.conf:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | pinot.broker.client.queryPort=8099
20 | pinot.broker.routing.table.builder.class=random
21 | pinot.set.instance.id.to.hostname=true
--------------------------------------------------------------------------------
/flink-connector-pinot/src/test/resources/pinot-controller/pinot-controller.conf:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | controller.helix.cluster.name=pinot
20 | controller.host=pinot-controller
21 | controller.port=9000
22 | controller.data.dir=/var/pinot/controller/data/data
23 | controller.local.temp.dir=/var/pinot/controller/data
24 | controller.zk.str=zookeeper:2181
25 | pinot.set.instance.id.to.hostname=true
--------------------------------------------------------------------------------
/flink-connector-pinot/src/test/resources/pinot-server/pinot-server.conf:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | pinot.server.netty.port=8098
20 | pinot.server.adminapi.port=8097
21 | pinot.server.instance.dataDir=/var/pinot/server/data/index
22 | pinot.server.instance.segmentTarDir=/var/pinot/server/data/segment
23 | pinot.set.instance.id.to.hostname=true
24 | pinot.server.grpc.enable=true
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/Util.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis.common;
18 |
19 | public class Util {
20 | public static void checkArgument(boolean condition, String message) {
21 | if(!condition) {
22 | throw new IllegalArgumentException(message);
23 | }
24 | }
25 | }
26 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/hanlder/FlinkJedisConfigHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.hanlder;
19 |
20 | import java.util.Map;
21 | import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase;
22 |
23 | /**
24 | * handler to create flink jedis config.
25 | */
26 | public interface FlinkJedisConfigHandler extends RedisHandler {
27 |
28 | /**
29 | * create flink jedis config use sepecified properties.
30 | * @param properties used to create flink jedis config
31 | * @return flink jedis config
32 | */
33 | FlinkJedisConfigBase createFlinkJedisConfig(Map properties);
34 | }
35 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/hanlder/RedisHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.hanlder;
19 |
20 | import java.io.Serializable;
21 | import java.util.Collections;
22 | import java.util.List;
23 | import java.util.Map;
24 |
25 | /***
26 | * redis handler to create redis mapper and flink jedis config.
27 | */
28 | public interface RedisHandler extends Serializable {
29 |
30 | /**
31 | * require context for spi to find this redis handler.
32 | * @return properties to find correct redis handler.
33 | */
34 | Map requiredContext();
35 |
36 | /**
37 | * suppport properties used for this redis handler.
38 | * @return support properties list
39 | * @throws Exception
40 | */
41 | default List supportProperties() throws Exception {
42 | return Collections.emptyList();
43 | }
44 |
45 | }
46 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/hanlder/RedisMapperHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.hanlder;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisMapper;
21 | import org.slf4j.Logger;
22 | import org.slf4j.LoggerFactory;
23 |
24 | import java.lang.reflect.Constructor;
25 | import java.util.Map;
26 |
27 | import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_KEY_TTL;
28 |
29 | /**
30 | * handler for create redis mapper.
31 | */
32 | public interface RedisMapperHandler extends RedisHandler {
33 |
34 | Logger LOGGER = LoggerFactory.getLogger(RedisMapperHandler.class);
35 |
36 | /**
37 | * create a correct redis mapper use properties.
38 | * @param properties to create redis mapper.
39 | * @return redis mapper.
40 | */
41 | default RedisMapper createRedisMapper(Map properties) {
42 | String ttl = properties.get(REDIS_KEY_TTL);
43 | try {
44 | Class redisMapper = Class.forName(this.getClass().getCanonicalName());
45 |
46 | if (ttl == null) {
47 | return (RedisMapper) redisMapper.newInstance();
48 | }
49 | Constructor c = redisMapper.getConstructor(Integer.class);
50 | return (RedisMapper) c.newInstance(Integer.parseInt(ttl));
51 | } catch (Exception e) {
52 | LOGGER.error("create redis mapper failed", e);
53 | throw new RuntimeException(e);
54 | }
55 | }
56 |
57 | }
58 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/DecrByExMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * decrease with expire operation redis mapper.
24 | */
25 | public class DecrByExMapper extends RowRedisMapper {
26 |
27 | public DecrByExMapper() {
28 | super(RedisCommand.DESCRBY_EX);
29 | }
30 |
31 | public DecrByExMapper(Integer ttl) {
32 | super(ttl, RedisCommand.DESCRBY_EX);
33 | }
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/DecrByMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * decrease operation redis mapper.
24 | */
25 | public class DecrByMapper extends RowRedisMapper {
26 |
27 | public DecrByMapper() {
28 | super(RedisCommand.DECRBY);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/HIncrByMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * HSET operation redis mapper.
24 | */
25 | public class HIncrByMapper extends RowRedisMapper {
26 |
27 | public HIncrByMapper() {
28 | super(RedisCommand.HINCRBY );
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/HSetMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * HSET operation redis mapper.
24 | */
25 | public class HSetMapper extends RowRedisMapper {
26 |
27 | public HSetMapper() {
28 | super(RedisCommand.HSET);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/IncrByExMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * Delta plus with expire key operation redis mapper.
24 | */
25 | public class IncrByExMapper extends RowRedisMapper {
26 |
27 | public IncrByExMapper() {
28 | super(RedisCommand.INCRBY_EX);
29 | }
30 |
31 | public IncrByExMapper(Integer ttl) {
32 | super(ttl, RedisCommand.INCRBY_EX);
33 | }
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/IncrByMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * Delta plus operation
24 | */
25 | public class IncrByMapper extends RowRedisMapper {
26 |
27 | public IncrByMapper() {
28 | super(RedisCommand.INCRBY);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/LPushMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * LPUSH operation redis mapper.
24 | */
25 | public class LPushMapper extends RowRedisMapper {
26 |
27 | public LPushMapper() {
28 | super(RedisCommand.LPUSH);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/PfAddMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * PFADD operation redis mapper.
24 | */
25 | public class PfAddMapper extends RowRedisMapper {
26 |
27 | public PfAddMapper() {
28 | super(RedisCommand.PFADD);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/RPushMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * RPUSH operation redis mapper.
24 | */
25 | public class RPushMapper extends RowRedisMapper {
26 |
27 | public RPushMapper() {
28 | super(RedisCommand.RPUSH);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/SAddMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * SADD operation redis mapper.
24 | */
25 | public class SAddMapper extends RowRedisMapper {
26 |
27 | public SAddMapper() {
28 | super(RedisCommand.SADD);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/SetExMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * SET with expire key operation redis mapper.
24 | */
25 | public class SetExMapper extends RowRedisMapper {
26 |
27 | public SetExMapper() {
28 | super(RedisCommand.SETEX);
29 | }
30 |
31 | public SetExMapper(Integer ttl) {
32 | super(ttl, RedisCommand.SETEX);
33 | }
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/SetMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
21 |
22 | /**
23 | * SET operation redis mapper.
24 | */
25 | public class SetMapper extends RowRedisMapper {
26 |
27 | public SetMapper() {
28 | super(RedisCommand.SET);
29 | }
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/ZAddMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 |
21 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
22 |
23 | /**
24 | * ZADD operation redis mapper.
25 | */
26 | public class ZAddMapper extends RowRedisMapper {
27 |
28 | public ZAddMapper() {
29 | super(RedisCommand.ZADD);
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/mapper/row/ZIncrByMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.common.mapper.row;
19 |
20 |
21 | import org.apache.flink.streaming.connectors.redis.common.mapper.RedisCommand;
22 |
23 | /**
24 | * ZINCRBY operation redis mapper.
25 | */
26 | public class ZIncrByMapper extends RowRedisMapper {
27 |
28 | public ZIncrByMapper() {
29 | super(RedisCommand.ZINCRBY);
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/descriptor/RedisValidator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.connectors.redis.descriptor;
19 |
20 | /**
21 | * redis validator for validate redis descriptor.
22 | */
23 | public class RedisValidator {
24 | public static final String REDIS = "redis";
25 | public static final String REDIS_MODE = "redis-mode";
26 | public static final String REDIS_NODES = "cluster-nodes";
27 | public static final String REDIS_CLUSTER = "cluster";
28 | public static final String REDIS_SENTINEL = "sentinel";
29 | public static final String REDIS_COMMAND = "command";
30 | public static final String REDIS_MASTER_NAME = "master.name";
31 | public static final String SENTINELS_INFO = "sentinels.info";
32 | public static final String SENTINELS_PASSWORD = "sentinels.password";
33 | public static final String REDIS_CLUSTER_PASSWORD = "cluster.password";
34 | public static final String REDIS_KEY_TTL = "key.ttl";
35 |
36 | }
37 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/resources/META-INF/services/org.apache.flink.streaming.connectors.redis.common.hanlder.RedisHandler:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.streaming.connectors.redis.common.mapper.row.SetExMapper
17 | org.apache.flink.streaming.connectors.redis.common.mapper.row.DecrByMapper
18 | org.apache.flink.streaming.connectors.redis.common.mapper.row.DecrByExMapper
19 | org.apache.flink.streaming.connectors.redis.common.mapper.row.HSetMapper
20 | org.apache.flink.streaming.connectors.redis.common.mapper.row.IncrByMapper
21 | org.apache.flink.streaming.connectors.redis.common.mapper.row.IncrByExMapper
22 | org.apache.flink.streaming.connectors.redis.common.mapper.row.LPushMapper
23 | org.apache.flink.streaming.connectors.redis.common.mapper.row.PfAddMapper
24 | org.apache.flink.streaming.connectors.redis.common.mapper.row.RPushMapper
25 | org.apache.flink.streaming.connectors.redis.common.mapper.row.SAddMapper
26 | org.apache.flink.streaming.connectors.redis.common.mapper.row.SetMapper
27 | org.apache.flink.streaming.connectors.redis.common.mapper.row.ZAddMapper
28 | org.apache.flink.streaming.connectors.redis.common.config.handler.FlinkJedisClusterConfigHandler
29 | org.apache.flink.streaming.connectors.redis.common.config.handler.FlinkJedisSentinelConfigHandler
--------------------------------------------------------------------------------
/flink-connector-redis/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | org.apache.flink.streaming.connectors.redis.RedisTableSinkFactory
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisITCaseBase.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis;
18 |
19 | import org.apache.flink.test.util.AbstractTestBase;
20 | import org.junit.AfterClass;
21 | import org.junit.BeforeClass;
22 | import redis.embedded.RedisServer;
23 |
24 | import java.io.IOException;
25 |
26 | import static org.apache.flink.util.NetUtils.getAvailablePort;
27 |
28 | public abstract class RedisITCaseBase extends AbstractTestBase {
29 |
30 | public static final int REDIS_PORT = getAvailablePort().getPort();
31 | public static final String REDIS_HOST = "127.0.0.1";
32 |
33 | private static RedisServer redisServer;
34 |
35 | @BeforeClass
36 | public static void createRedisServer() throws IOException, InterruptedException {
37 | redisServer = new RedisServer(REDIS_PORT);
38 | redisServer.start();
39 | }
40 |
41 | @AfterClass
42 | public static void stopRedisServer(){
43 | redisServer.stop();
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis.common.config;
18 |
19 | import org.apache.flink.util.TestLogger;
20 | import org.junit.Test;
21 |
22 | public class FlinkJedisConfigBaseTest extends TestLogger {
23 |
24 | @Test(expected = IllegalArgumentException.class)
25 | public void shouldThrowIllegalArgumentExceptionIfTimeOutIsNegative(){
26 | new TestConfig(-1, 0, 0, 0, false, false, false);
27 | }
28 |
29 | @Test(expected = IllegalArgumentException.class)
30 | public void shouldThrowIllegalArgumentExceptionIfMaxTotalIsNegative(){
31 | new TestConfig(1, -1, 0, 0, false, false, false);
32 | }
33 |
34 | @Test(expected = IllegalArgumentException.class)
35 | public void shouldThrowIllegalArgumentExceptionIfMaxIdleIsNegative(){
36 | new TestConfig(0, 0, -1, 0, false, false, false);
37 | }
38 |
39 | @Test(expected = IllegalArgumentException.class)
40 | public void shouldThrowIllegalArgumentExceptionIfMinIdleIsNegative(){
41 | new TestConfig(0, 0, 0, -1, false, false, false);
42 | }
43 |
44 | private class TestConfig extends FlinkJedisConfigBase {
45 | protected TestConfig(int connectionTimeout, int maxTotal, int maxIdle, int minIdle,
46 | boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) {
47 | super(connectionTimeout, maxTotal, maxIdle, minIdle, "dummy", false, testOnBorrow, testOnReturn, testWhileIdle);
48 | }
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisPoolConfigTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis.common.config;
18 |
19 | import org.apache.flink.util.TestLogger;
20 | import org.junit.Test;
21 |
22 | public class JedisPoolConfigTest extends TestLogger {
23 |
24 | @Test(expected = NullPointerException.class)
25 | public void shouldThrowNullPointExceptionIfHostValueIsNull(){
26 | FlinkJedisPoolConfig.Builder builder = new FlinkJedisPoolConfig.Builder();
27 | builder.build();
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisSentinelConfigTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis.common.config;
18 |
19 | import org.apache.flink.util.TestLogger;
20 | import org.junit.Test;
21 |
22 | import java.util.HashSet;
23 | import java.util.Set;
24 |
25 | public class JedisSentinelConfigTest extends TestLogger {
26 |
27 | public static final String MASTER_NAME = "test-master";
28 |
29 | @Test(expected = NullPointerException.class)
30 | public void shouldThrowNullPointExceptionIfMasterValueIsNull(){
31 | FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
32 | Set sentinels = new HashSet<>();
33 | sentinels.add("127.0.0.1");
34 | builder.setSentinels(sentinels).build();
35 | }
36 |
37 | @Test(expected = NullPointerException.class)
38 | public void shouldThrowNullPointExceptionIfSentinelsValueIsNull(){
39 | FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
40 | builder.setMasterName(MASTER_NAME).build();
41 | }
42 |
43 | @Test(expected = IllegalArgumentException.class)
44 | public void shouldThrowNullPointExceptionIfSentinelsValueIsEmpty(){
45 | FlinkJedisSentinelConfig.Builder builder = new FlinkJedisSentinelConfig.Builder();
46 | Set sentinels = new HashSet<>();
47 | builder.setMasterName(MASTER_NAME).setSentinels(sentinels).build();
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/mapper/RedisDataTypeDescriptionTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 | package org.apache.flink.streaming.connectors.redis.common.mapper;
18 |
19 | import org.apache.flink.streaming.connectors.redis.RedisSinkITCase;
20 | import org.apache.flink.util.TestLogger;
21 | import org.junit.Test;
22 |
23 | import static org.junit.Assert.assertEquals;
24 | import static org.junit.Assert.assertNull;
25 |
26 | public class RedisDataTypeDescriptionTest extends TestLogger {
27 |
28 | @Test(expected=IllegalArgumentException.class)
29 | public void shouldThrowExceptionIfAdditionalKeyIsNotGivenForHashDataType(){
30 | RedisSinkITCase.RedisCommandMapper redisCommandMapper = new RedisSinkITCase.RedisCommandMapper(RedisCommand.HSET);
31 | redisCommandMapper.getCommandDescription();
32 | }
33 |
34 | @Test(expected=IllegalArgumentException.class)
35 | public void shouldThrowExceptionIfAdditionalTTLIsNotGivenForStringDataTypeWithTTL(){
36 | RedisSinkITCase.RedisCommandMapper redisCommandMapper = new RedisSinkITCase.RedisCommandMapper(RedisCommand.SETEX);
37 | redisCommandMapper.getCommandDescription();
38 | }
39 |
40 | @Test
41 | public void shouldReturnNullForAdditionalDataType(){
42 | RedisSinkITCase.RedisCommandMapper redisCommandMapper = new RedisSinkITCase.RedisCommandMapper(RedisCommand.LPUSH);
43 | RedisCommandDescription redisDataTypeDescription = redisCommandMapper.getCommandDescription();
44 | assertEquals(RedisDataType.LIST, redisDataTypeDescription.getCommand().getRedisDataType());
45 | assertNull(redisDataTypeDescription.getAdditionalKey());
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/flink-connector-redis/src/test/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 |
19 | # This file ensures that tests executed from the IDE show log output
20 |
21 | log4j.rootLogger=INFO, console
22 |
23 | # Log all infos in the given file
24 | log4j.appender.console=org.apache.log4j.ConsoleAppender
25 | log4j.appender.console.target=System.err
26 | log4j.appender.console.layout=org.apache.log4j.PatternLayout
27 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
28 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/main/java/org/apache/flink/streaming/siddhi/exception/DuplicatedStreamException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.exception;
19 |
20 | public class DuplicatedStreamException extends RuntimeException {
21 | public DuplicatedStreamException(String message) {
22 | super(message);
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/main/java/org/apache/flink/streaming/siddhi/exception/UndefinedStreamException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.exception;
19 |
20 | public class UndefinedStreamException extends RuntimeException {
21 | public UndefinedStreamException(String message) {
22 | super(message);
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/main/java/org/apache/flink/streaming/siddhi/operator/StreamRecordComparator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.operator;
19 |
20 | import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
21 |
22 | import java.io.Serializable;
23 | import java.util.Comparator;
24 |
25 | /**
26 | * Stream Record Timestamp Comparator
27 | */
28 | public class StreamRecordComparator implements Comparator>, Serializable {
29 | private static final long serialVersionUID = 1581054988433915305L;
30 |
31 | @Override
32 | public int compare(StreamRecord o1, StreamRecord o2) {
33 | if (o1.getTimestamp() < o2.getTimestamp()) {
34 | return -1;
35 | } else if (o1.getTimestamp() > o2.getTimestamp()) {
36 | return 1;
37 | } else {
38 | return 0;
39 | }
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/main/java/org/apache/flink/streaming/siddhi/utils/SiddhiStreamFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.utils;
19 |
20 | import org.apache.flink.api.java.tuple.Tuple2;
21 | import org.apache.flink.streaming.api.datastream.DataStream;
22 | import org.apache.flink.streaming.siddhi.operator.SiddhiOperatorContext;
23 | import org.apache.flink.streaming.siddhi.operator.SiddhiStreamOperator;
24 |
25 | /**
26 | * Convert SiddhiCEPExecutionPlan to SiddhiCEP Operator and build output DataStream
27 | */
28 | public class SiddhiStreamFactory {
29 | @SuppressWarnings("unchecked")
30 | public static DataStream createDataStream(SiddhiOperatorContext context, DataStream> namedStream, String outStreamId) {
31 | return namedStream.transform(context.getName(), context.getOutputStreamType(), new SiddhiStreamOperator(context,outStreamId));
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/test/java/org/apache/flink/streaming/siddhi/schema/StreamSerializerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.schema;
19 |
20 | import org.apache.flink.api.java.typeutils.TypeExtractor;
21 | import org.apache.flink.streaming.siddhi.source.Event;
22 | import org.junit.Assert;
23 | import org.junit.Test;
24 |
25 | public class StreamSerializerTest {
26 | private static final long CURRENT = System.currentTimeMillis();
27 |
28 | @Test
29 | public void testSimplePojoRead() {
30 | Event event = new Event();
31 | event.setId(1);
32 | event.setName("test");
33 | event.setPrice(56.7);
34 | event.setTimestamp(CURRENT);
35 |
36 | StreamSchema schema = new StreamSchema<>(TypeExtractor.createTypeInfo(Event.class), "id", "name", "price", "timestamp");
37 | StreamSerializer reader = new StreamSerializer<>(schema);
38 | Assert.assertArrayEquals(new Object[]{1, "test", 56.7, CURRENT}, reader.getRow(event));
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/test/java/org/apache/flink/streaming/siddhi/utils/SiddhiTupleFactoryTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.flink.streaming.siddhi.utils;
19 |
20 | import org.apache.flink.api.java.tuple.Tuple5;
21 | import org.junit.Test;
22 |
23 | import static org.junit.Assert.assertArrayEquals;
24 | import static org.junit.Assert.assertEquals;
25 |
26 | public class SiddhiTupleFactoryTest {
27 | @Test
28 | public void testConvertObjectArrayToTuple() {
29 | Object[] row = new Object[]{1, "message", 1234567L, true, new Object()};
30 | Tuple5 tuple5 = SiddhiTupleFactory.newTuple(row);
31 | assertEquals(5, tuple5.getArity());
32 | assertArrayEquals(row, new Object[]{
33 | tuple5.f0,
34 | tuple5.f1,
35 | tuple5.f2,
36 | tuple5.f3,
37 | tuple5.f4
38 | });
39 | }
40 |
41 | @Test(expected = IllegalArgumentException.class)
42 | public void testConvertTooLongObjectArrayToTuple() {
43 | Object[] row = new Object[26];
44 | SiddhiTupleFactory.newTuple(row);
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/test/resources/log4j-test.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | #
18 |
19 | # Set root logger level to OFF to not flood build logs
20 | # set manually to INFO for debugging purposes
21 | log4j.rootLogger=INFO, testlogger
22 |
23 | # A1 is set to be a ConsoleAppender.
24 | log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
25 | log4j.appender.testlogger.target = System.err
26 | log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
27 | log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
28 |
--------------------------------------------------------------------------------
/flink-library-siddhi/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
18 |
19 |
20 |
21 |
22 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n
23 |
24 |
25 |
26 |
27 |
28 |
29 |
30 |
31 |
32 |
33 |
34 |
35 |
--------------------------------------------------------------------------------