53 | *
54 | * @param Type of the elements handled by this sink
55 | */
56 | @PublicEvolving
57 | public class MongoSink implements Sink {
58 |
59 | private static final long serialVersionUID = 1L;
60 |
61 | private final MongoConnectionOptions connectionOptions;
62 | private final MongoWriteOptions writeOptions;
63 | private final MongoSerializationSchema serializationSchema;
64 |
65 | MongoSink(
66 | MongoConnectionOptions connectionOptions,
67 | MongoWriteOptions writeOptions,
68 | MongoSerializationSchema serializationSchema) {
69 | this.connectionOptions = checkNotNull(connectionOptions);
70 | this.writeOptions = checkNotNull(writeOptions);
71 | this.serializationSchema = checkNotNull(serializationSchema);
72 | ClosureCleaner.clean(
73 | serializationSchema, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
74 | }
75 |
76 | public static MongoSinkBuilder builder() {
77 | return new MongoSinkBuilder<>();
78 | }
79 |
80 | @Override
81 | public SinkWriter createWriter(InitContext context) {
82 | return new MongoWriter<>(
83 | connectionOptions,
84 | writeOptions,
85 | writeOptions.getDeliveryGuarantee() == DeliveryGuarantee.AT_LEAST_ONCE,
86 | context,
87 | serializationSchema);
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/MongoSinkBuilder.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.connector.mongodb.sink;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.connector.base.DeliveryGuarantee;
22 | import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
23 | import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
24 | import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
25 | import org.apache.flink.util.InstantiationUtil;
26 |
27 | import com.mongodb.client.model.WriteModel;
28 |
29 | import static org.apache.flink.util.Preconditions.checkNotNull;
30 | import static org.apache.flink.util.Preconditions.checkState;
31 |
32 | /**
33 | * Base builder to construct a {@link MongoSink}.
34 | *
35 | * @param type of the records converted to MongoDB bulk request
36 | */
37 | @PublicEvolving
38 | public class MongoSinkBuilder {
39 |
40 | private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder;
41 | private final MongoWriteOptions.MongoWriteOptionsBuilder writeOptionsBuilder;
42 |
43 | private MongoSerializationSchema serializationSchema;
44 |
45 | MongoSinkBuilder() {
46 | this.connectionOptionsBuilder = MongoConnectionOptions.builder();
47 | this.writeOptionsBuilder = MongoWriteOptions.builder();
48 | }
49 |
50 | /**
51 | * Sets the connection string of MongoDB.
52 | *
53 | * @param uri connection string of MongoDB
54 | * @return this builder
55 | */
56 | public MongoSinkBuilder setUri(String uri) {
57 | connectionOptionsBuilder.setUri(uri);
58 | return this;
59 | }
60 |
61 | /**
62 | * Sets the database to sink of MongoDB.
63 | *
64 | * @param database the database to sink of MongoDB.
65 | * @return this builder
66 | */
67 | public MongoSinkBuilder setDatabase(String database) {
68 | connectionOptionsBuilder.setDatabase(database);
69 | return this;
70 | }
71 |
72 | /**
73 | * Sets the collection to sink of MongoDB.
74 | *
75 | * @param collection the collection to sink of MongoDB.
76 | * @return this builder
77 | */
78 | public MongoSinkBuilder setCollection(String collection) {
79 | connectionOptionsBuilder.setCollection(collection);
80 | return this;
81 | }
82 |
83 | /**
84 | * Sets the maximum number of actions to buffer for each batch request. You can pass -1 to
85 | * disable batching.
86 | *
87 | * @param batchSize the maximum number of actions to buffer for each batch request.
88 | * @return this builder
89 | */
90 | public MongoSinkBuilder setBatchSize(int batchSize) {
91 | writeOptionsBuilder.setBatchSize(batchSize);
92 | return this;
93 | }
94 |
95 | /**
96 | * Sets the batch flush interval, in milliseconds. You can pass -1 to disable it.
97 | *
98 | * @param batchIntervalMs the batch flush interval, in milliseconds.
99 | * @return this builder
100 | */
101 | public MongoSinkBuilder setBatchIntervalMs(long batchIntervalMs) {
102 | writeOptionsBuilder.setBatchIntervalMs(batchIntervalMs);
103 | return this;
104 | }
105 |
106 | /**
107 | * Sets the max retry times if writing records failed.
108 | *
109 | * @param maxRetries the max retry times.
110 | * @return this builder
111 | */
112 | public MongoSinkBuilder setMaxRetries(int maxRetries) {
113 | writeOptionsBuilder.setMaxRetries(maxRetries);
114 | return this;
115 | }
116 |
117 | /**
118 | * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
119 | * DeliveryGuarantee#AT_LEAST_ONCE}
120 | *
121 | * @param deliveryGuarantee which describes the record emission behaviour
122 | * @return this builder
123 | */
124 | public MongoSinkBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
125 | writeOptionsBuilder.setDeliveryGuarantee(deliveryGuarantee);
126 | return this;
127 | }
128 |
129 | /**
130 | * Sets the serialization schema which is invoked on every record to convert it to MongoDB bulk
131 | * request.
132 | *
133 | * @param serializationSchema to process records into MongoDB bulk {@link WriteModel}.
134 | * @return this builder
135 | */
136 | public MongoSinkBuilder setSerializationSchema(
137 | MongoSerializationSchema serializationSchema) {
138 | checkNotNull(serializationSchema);
139 | checkState(
140 | InstantiationUtil.isSerializable(serializationSchema),
141 | "The mongo serialization schema must be serializable.");
142 | this.serializationSchema = serializationSchema;
143 | return this;
144 | }
145 |
146 | /**
147 | * Constructs the {@link MongoSink} with the properties configured this builder.
148 | *
149 | * @return {@link MongoSink}
150 | */
151 | public MongoSink build() {
152 | checkNotNull(serializationSchema, "The serialization schema must be supplied");
153 | return new MongoSink<>(
154 | connectionOptionsBuilder.build(), writeOptionsBuilder.build(), serializationSchema);
155 | }
156 | }
157 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/context/DefaultMongoSinkContext.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.connector.mongodb.sink.writer.context;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.api.connector.sink2.Sink;
22 | import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
23 |
24 | /** Default {@link MongoSinkContext} implementation. */
25 | @Internal
26 | public class DefaultMongoSinkContext implements MongoSinkContext {
27 |
28 | private final Sink.InitContext initContext;
29 | private final MongoWriteOptions writeOptions;
30 |
31 | public DefaultMongoSinkContext(Sink.InitContext initContext, MongoWriteOptions writeOptions) {
32 | this.initContext = initContext;
33 | this.writeOptions = writeOptions;
34 | }
35 |
36 | @Override
37 | public Sink.InitContext getInitContext() {
38 | return initContext;
39 | }
40 |
41 | @Override
42 | public long processTime() {
43 | return initContext.getProcessingTimeService().getCurrentProcessingTime();
44 | }
45 |
46 | @Override
47 | public MongoWriteOptions getWriteOptions() {
48 | return writeOptions;
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/context/MongoSinkContext.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.connector.mongodb.sink.writer.context;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.api.connector.sink2.Sink;
22 | import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
23 | import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
24 |
25 | /** This context provides information for {@link MongoSerializationSchema}. */
26 | @PublicEvolving
27 | public interface MongoSinkContext {
28 |
29 | /** Returns the current sink's init context. */
30 | Sink.InitContext getInitContext();
31 |
32 | /** Returns the current process time in flink. */
33 | long processTime();
34 |
35 | /** Returns the write options of MongoSink. */
36 | MongoWriteOptions getWriteOptions();
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.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.connector.mongodb.sink.writer.serializer;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.api.common.serialization.SerializationSchema;
22 | import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
23 | import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
24 |
25 | import com.mongodb.client.model.WriteModel;
26 | import org.bson.BsonDocument;
27 |
28 | import java.io.Serializable;
29 |
30 | /**
31 | * The serialization schema for how to serialize records into MongoDB.
32 | *
33 | * @param The message type send to MongoDB.
34 | */
35 | @PublicEvolving
36 | public interface MongoSerializationSchema extends Serializable {
37 |
38 | /**
39 | * Initialization method for the schema. It is called before the actual working methods {@link
40 | * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
41 | *
42 | *
The provided {@link SerializationSchema.InitializationContext} can be used to access
43 | * additional features such as registering user metrics.
44 | *
45 | * @param initializationContext Contextual information that can be used during initialization.
46 | * @param sinkContext Runtime information i.e. partitions, subtaskId.
47 | * @param sinkConfiguration All the configure options for the MongoDB sink.
48 | */
49 | default void open(
50 | SerializationSchema.InitializationContext initializationContext,
51 | MongoSinkContext sinkContext,
52 | MongoWriteOptions sinkConfiguration)
53 | throws Exception {
54 | // Nothing to do by default.
55 | }
56 |
57 | /**
58 | * Serializes the given element into {@link WriteModel}.
59 | *
60 | * @param element Element to be serialized.
61 | * @param sinkContext Context to provide extra information.
62 | */
63 | WriteModel serialize(IN element, MongoSinkContext sinkContext);
64 | }
65 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.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.connector.mongodb.source.enumerator;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner;
22 | import org.apache.flink.connector.mongodb.source.reader.split.MongoSourceSplitReader;
23 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
24 |
25 | import java.util.ArrayList;
26 | import java.util.HashMap;
27 | import java.util.List;
28 | import java.util.Map;
29 |
30 | /**
31 | * The state class for MongoDB source enumerator, used for storing the split state. This class is
32 | * managed and controlled by {@link MongoSplitAssigner}.
33 | */
34 | @PublicEvolving
35 | public class MongoSourceEnumState {
36 |
37 | /** The Mongo collections remaining. */
38 | private final List remainingCollections;
39 |
40 | /**
41 | * The paths that are no longer in the enumerator checkpoint, but have been processed before.
42 | */
43 | private final List alreadyProcessedCollections;
44 |
45 | /** The scan splits in the checkpoint. */
46 | private final List remainingScanSplits;
47 |
48 | /**
49 | * The scan splits that the {@link MongoSourceEnumerator} has assigned to {@link
50 | * MongoSourceSplitReader}s.
51 | */
52 | private final Map assignedScanSplits;
53 |
54 | /** The pipeline has been triggered and topic partitions have been assigned to readers. */
55 | private final boolean initialized;
56 |
57 | public MongoSourceEnumState(
58 | List remainingCollections,
59 | List alreadyProcessedCollections,
60 | List remainingScanSplits,
61 | Map assignedScanSplits,
62 | boolean initialized) {
63 | this.remainingCollections = remainingCollections;
64 | this.alreadyProcessedCollections = alreadyProcessedCollections;
65 | this.remainingScanSplits = remainingScanSplits;
66 | this.assignedScanSplits = assignedScanSplits;
67 | this.initialized = initialized;
68 | }
69 |
70 | public List getRemainingCollections() {
71 | return remainingCollections;
72 | }
73 |
74 | public List getAlreadyProcessedCollections() {
75 | return alreadyProcessedCollections;
76 | }
77 |
78 | public List getRemainingScanSplits() {
79 | return remainingScanSplits;
80 | }
81 |
82 | public Map getAssignedScanSplits() {
83 | return assignedScanSplits;
84 | }
85 |
86 | public boolean isInitialized() {
87 | return initialized;
88 | }
89 |
90 | /** The initial assignment state for Mongo. */
91 | public static MongoSourceEnumState initialState() {
92 | return new MongoSourceEnumState(
93 | new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), new HashMap<>(), false);
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.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.connector.mongodb.source.enumerator;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
22 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
23 | import org.apache.flink.core.io.SimpleVersionedSerializer;
24 |
25 | import java.io.ByteArrayInputStream;
26 | import java.io.ByteArrayOutputStream;
27 | import java.io.DataInput;
28 | import java.io.DataInputStream;
29 | import java.io.DataOutputStream;
30 | import java.io.IOException;
31 | import java.util.List;
32 | import java.util.Map;
33 |
34 | import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeList;
35 | import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap;
36 | import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeList;
37 | import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap;
38 | import static org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer.SCAN_SPLIT_FLAG;
39 |
40 | /** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Mongo source. */
41 | @Internal
42 | public class MongoSourceEnumStateSerializer
43 | implements SimpleVersionedSerializer {
44 |
45 | public static final MongoSourceEnumStateSerializer INSTANCE =
46 | new MongoSourceEnumStateSerializer();
47 |
48 | private MongoSourceEnumStateSerializer() {
49 | // Singleton instance.
50 | }
51 |
52 | @Override
53 | public int getVersion() {
54 | // We use MongoSourceSplitSerializer's version because we use reuse this class.
55 | return MongoSourceSplitSerializer.CURRENT_VERSION;
56 | }
57 |
58 | @Override
59 | public byte[] serialize(MongoSourceEnumState state) throws IOException {
60 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
61 | DataOutputStream out = new DataOutputStream(baos)) {
62 | serializeList(out, state.getRemainingCollections(), DataOutputStream::writeUTF);
63 |
64 | serializeList(out, state.getAlreadyProcessedCollections(), DataOutputStream::writeUTF);
65 |
66 | serializeList(
67 | out,
68 | state.getRemainingScanSplits(),
69 | MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
70 |
71 | serializeMap(
72 | out,
73 | state.getAssignedScanSplits(),
74 | DataOutputStream::writeUTF,
75 | MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
76 |
77 | out.writeBoolean(state.isInitialized());
78 |
79 | out.flush();
80 | return baos.toByteArray();
81 | }
82 | }
83 |
84 | @Override
85 | public MongoSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
86 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
87 | DataInputStream in = new DataInputStream(bais)) {
88 | List remainingCollections = deserializeList(in, DataInput::readUTF);
89 | List alreadyProcessedCollections = deserializeList(in, DataInput::readUTF);
90 | List remainingScanSplits =
91 | deserializeList(in, i -> deserializeMongoScanSourceSplit(version, i));
92 |
93 | Map assignedScanSplits =
94 | deserializeMap(
95 | in,
96 | DataInput::readUTF,
97 | i -> deserializeMongoScanSourceSplit(version, i));
98 |
99 | boolean initialized = in.readBoolean();
100 |
101 | return new MongoSourceEnumState(
102 | remainingCollections,
103 | alreadyProcessedCollections,
104 | remainingScanSplits,
105 | assignedScanSplits,
106 | initialized);
107 | }
108 | }
109 |
110 | private static MongoScanSourceSplit deserializeMongoScanSourceSplit(
111 | int version, DataInputStream in) throws IOException {
112 | int splitKind = in.readInt();
113 | if (splitKind == SCAN_SPLIT_FLAG) {
114 | return MongoSourceSplitSerializer.INSTANCE.deserializeMongoScanSourceSplit(version, in);
115 | }
116 | throw new IOException("Split kind mismatch expect 1 but was " + splitKind);
117 | }
118 | }
119 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.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.connector.mongodb.source.enumerator;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.api.connector.source.Boundedness;
22 | import org.apache.flink.api.connector.source.SplitEnumerator;
23 | import org.apache.flink.api.connector.source.SplitEnumeratorContext;
24 | import org.apache.flink.connector.mongodb.source.MongoSource;
25 | import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner;
26 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
27 |
28 | import org.slf4j.Logger;
29 | import org.slf4j.LoggerFactory;
30 |
31 | import javax.annotation.Nullable;
32 |
33 | import java.io.IOException;
34 | import java.util.Iterator;
35 | import java.util.List;
36 | import java.util.Optional;
37 | import java.util.TreeSet;
38 |
39 | /** The enumerator class for {@link MongoSource}. */
40 | @Internal
41 | public class MongoSourceEnumerator
42 | implements SplitEnumerator {
43 |
44 | private static final Logger LOG = LoggerFactory.getLogger(MongoSourceEnumerator.class);
45 |
46 | private final Boundedness boundedness;
47 | private final SplitEnumeratorContext context;
48 | private final MongoSplitAssigner splitAssigner;
49 | private final TreeSet readersAwaitingSplit;
50 |
51 | public MongoSourceEnumerator(
52 | Boundedness boundedness,
53 | SplitEnumeratorContext context,
54 | MongoSplitAssigner splitAssigner) {
55 | this.boundedness = boundedness;
56 | this.context = context;
57 | this.splitAssigner = splitAssigner;
58 | this.readersAwaitingSplit = new TreeSet<>();
59 | }
60 |
61 | @Override
62 | public void start() {
63 | splitAssigner.open();
64 | }
65 |
66 | @Override
67 | public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
68 | if (!context.registeredReaders().containsKey(subtaskId)) {
69 | // reader failed between sending the request and now. skip this request.
70 | return;
71 | }
72 |
73 | readersAwaitingSplit.add(subtaskId);
74 | assignSplits();
75 | }
76 |
77 | @Override
78 | public void addSplitsBack(List splits, int subtaskId) {
79 | LOG.debug("Mongo Source Enumerator adds splits back: {}", splits);
80 | splitAssigner.addSplitsBack(splits);
81 | }
82 |
83 | @Override
84 | public void addReader(int subtaskId) {
85 | LOG.debug("Adding reader {} to MongoSourceEnumerator.", subtaskId);
86 | }
87 |
88 | private void assignSplits() {
89 | final Iterator awaitingReader = readersAwaitingSplit.iterator();
90 |
91 | while (awaitingReader.hasNext()) {
92 | int nextAwaiting = awaitingReader.next();
93 | // if the reader that requested another split has failed in the meantime, remove
94 | // it from the list of waiting readers
95 | if (!context.registeredReaders().containsKey(nextAwaiting)) {
96 | awaitingReader.remove();
97 | continue;
98 | }
99 |
100 | // close idle readers
101 | if (splitAssigner.noMoreSplits() && boundedness == Boundedness.BOUNDED) {
102 | context.signalNoMoreSplits(nextAwaiting);
103 | awaitingReader.remove();
104 | LOG.info(
105 | "All scan splits have been assigned, closing idle reader {}", nextAwaiting);
106 | continue;
107 | }
108 |
109 | Optional split = splitAssigner.getNext();
110 | if (split.isPresent()) {
111 | final MongoSourceSplit mongoSplit = split.get();
112 | context.assignSplit(mongoSplit, nextAwaiting);
113 | awaitingReader.remove();
114 | LOG.info("Assign split {} to subtask {}", mongoSplit, nextAwaiting);
115 | break;
116 | } else {
117 | // there is no available splits by now, skip assigning
118 | break;
119 | }
120 | }
121 | }
122 |
123 | @Override
124 | public MongoSourceEnumState snapshotState(long checkpointId) {
125 | return splitAssigner.snapshotState(checkpointId);
126 | }
127 |
128 | @Override
129 | public void close() throws IOException {
130 | splitAssigner.close();
131 | }
132 | }
133 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.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.connector.mongodb.source.enumerator.assigner;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
22 | import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
23 | import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState;
24 | import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitters;
25 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
26 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
27 |
28 | import com.mongodb.MongoNamespace;
29 | import com.mongodb.client.MongoClient;
30 | import com.mongodb.client.MongoClients;
31 | import org.slf4j.Logger;
32 | import org.slf4j.LoggerFactory;
33 |
34 | import java.io.IOException;
35 | import java.util.Collection;
36 | import java.util.LinkedList;
37 | import java.util.List;
38 | import java.util.Map;
39 | import java.util.Optional;
40 |
41 | import static org.apache.flink.util.Preconditions.checkState;
42 |
43 | /** The split assigner for {@link MongoScanSourceSplit}. */
44 | @Internal
45 | public class MongoScanSplitAssigner implements MongoSplitAssigner {
46 |
47 | private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
48 |
49 | private final MongoConnectionOptions connectionOptions;
50 | private final MongoReadOptions readOptions;
51 |
52 | private final LinkedList remainingCollections;
53 | private final List alreadyProcessedCollections;
54 | private final LinkedList remainingScanSplits;
55 | private final Map assignedScanSplits;
56 | private boolean initialized;
57 |
58 | private MongoClient mongoClient;
59 |
60 | public MongoScanSplitAssigner(
61 | MongoConnectionOptions connectionOptions,
62 | MongoReadOptions readOptions,
63 | MongoSourceEnumState sourceEnumState) {
64 | this.connectionOptions = connectionOptions;
65 | this.readOptions = readOptions;
66 | this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());
67 | this.alreadyProcessedCollections = sourceEnumState.getAlreadyProcessedCollections();
68 | this.remainingScanSplits = new LinkedList<>(sourceEnumState.getRemainingScanSplits());
69 | this.assignedScanSplits = sourceEnumState.getAssignedScanSplits();
70 | this.initialized = sourceEnumState.isInitialized();
71 | }
72 |
73 | @Override
74 | public void open() {
75 | LOG.info("Mongo scan split assigner is opening.");
76 | if (!initialized) {
77 | String collectionId =
78 | String.format(
79 | "%s.%s",
80 | connectionOptions.getDatabase(), connectionOptions.getCollection());
81 | remainingCollections.add(collectionId);
82 | mongoClient = MongoClients.create(connectionOptions.getUri());
83 | initialized = true;
84 | }
85 | }
86 |
87 | @Override
88 | public Optional getNext() {
89 | if (!remainingScanSplits.isEmpty()) {
90 | // return remaining splits firstly
91 | MongoScanSourceSplit split = remainingScanSplits.poll();
92 | assignedScanSplits.put(split.splitId(), split);
93 | return Optional.of(split);
94 | } else {
95 | // it's turn for next collection
96 | String nextCollection = remainingCollections.poll();
97 | if (nextCollection != null) {
98 | // split the given collection into chunks (scan splits)
99 | Collection splits =
100 | MongoSplitters.split(
101 | mongoClient, readOptions, new MongoNamespace(nextCollection));
102 | remainingScanSplits.addAll(splits);
103 | alreadyProcessedCollections.add(nextCollection);
104 | return getNext();
105 | } else {
106 | return Optional.empty();
107 | }
108 | }
109 | }
110 |
111 | @Override
112 | public void addSplitsBack(Collection splits) {
113 | for (MongoSourceSplit split : splits) {
114 | if (split instanceof MongoScanSourceSplit) {
115 | remainingScanSplits.add((MongoScanSourceSplit) split);
116 | // we should remove the add-backed splits from the assigned list,
117 | // because they are failed
118 | assignedScanSplits.remove(split.splitId());
119 | }
120 | }
121 | }
122 |
123 | @Override
124 | public MongoSourceEnumState snapshotState(long checkpointId) {
125 | return new MongoSourceEnumState(
126 | remainingCollections,
127 | alreadyProcessedCollections,
128 | remainingScanSplits,
129 | assignedScanSplits,
130 | initialized);
131 | }
132 |
133 | @Override
134 | public boolean noMoreSplits() {
135 | checkState(initialized, "The noMoreSplits method was called but not initialized.");
136 | return remainingCollections.isEmpty() && remainingScanSplits.isEmpty();
137 | }
138 |
139 | @Override
140 | public void close() throws IOException {
141 | if (mongoClient != null) {
142 | mongoClient.close();
143 | LOG.info("Mongo scan split assigner is closed.");
144 | }
145 | }
146 | }
147 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.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.connector.mongodb.source.enumerator.assigner;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState;
22 | import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumerator;
23 | import org.apache.flink.connector.mongodb.source.reader.split.MongoSourceSplitReader;
24 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
25 |
26 | import java.io.IOException;
27 | import java.util.Collection;
28 | import java.util.Optional;
29 |
30 | /** The split assigner for {@link MongoSourceSplit}. */
31 | @Internal
32 | public interface MongoSplitAssigner {
33 |
34 | /**
35 | * Called to open the assigner to acquire any resources, like threads or network connections.
36 | */
37 | void open();
38 |
39 | /**
40 | * Called to close the assigner, in case it holds on to any resources, like threads or network
41 | * connections.
42 | */
43 | void close() throws IOException;
44 |
45 | /**
46 | * Gets the next split to assign to {@link MongoSourceSplitReader} when {@link
47 | * MongoSourceEnumerator} receives a split request, until there are {@link #noMoreSplits()}.
48 | */
49 | Optional getNext();
50 |
51 | /**
52 | * Adds a set of splits to this assigner. This happens for example when some split processing
53 | * failed and the splits need to be re-added.
54 | */
55 | void addSplitsBack(Collection splits);
56 |
57 | /** Snapshot the current assign state into checkpoint. */
58 | MongoSourceEnumState snapshotState(long checkpointId);
59 |
60 | /** Return whether there are no more splits. */
61 | boolean noMoreSplits();
62 | }
63 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoPaginationSplitter.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.connector.mongodb.source.enumerator.splitter;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
22 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
23 |
24 | import com.mongodb.MongoNamespace;
25 | import com.mongodb.client.model.Aggregates;
26 | import com.mongodb.client.model.Projections;
27 | import com.mongodb.client.model.Sorts;
28 | import org.bson.BsonDocument;
29 | import org.bson.conversions.Bson;
30 | import org.slf4j.Logger;
31 | import org.slf4j.LoggerFactory;
32 |
33 | import java.util.ArrayList;
34 | import java.util.Collection;
35 | import java.util.List;
36 |
37 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_BOUNDARY;
38 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_BOUNDARY;
39 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
40 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
41 |
42 | /** Mongo Splitter that splits MongoDB collection evenly by record counts. */
43 | @Internal
44 | public class MongoPaginationSplitter {
45 |
46 | private static final Logger LOG = LoggerFactory.getLogger(MongoPaginationSplitter.class);
47 |
48 | public static Collection split(MongoSplitContext splitContext) {
49 | MongoReadOptions readOptions = splitContext.getReadOptions();
50 | MongoNamespace namespace = splitContext.getMongoNamespace();
51 |
52 | // If partition record size isn't present, we'll use the partition size option and average
53 | // object size to calculate number of records in each partitioned split.
54 | Integer partitionRecordSize = readOptions.getPartitionRecordSize();
55 | if (partitionRecordSize == null) {
56 | long avgObjSizeInBytes = splitContext.getAvgObjSize();
57 | if (avgObjSizeInBytes == 0) {
58 | LOG.info(
59 | "{} seems to be an empty collection, Returning a single partition.",
60 | namespace);
61 | return MongoSingleSplitter.split(splitContext);
62 | }
63 |
64 | partitionRecordSize =
65 | Math.toIntExact(readOptions.getPartitionSize().getBytes() / avgObjSizeInBytes);
66 | }
67 |
68 | long totalNumOfDocuments = splitContext.getCount();
69 |
70 | if (partitionRecordSize >= totalNumOfDocuments) {
71 | LOG.info(
72 | "Fewer documents ({}) than the number of documents per partition ({}), Returning a single partition.",
73 | totalNumOfDocuments,
74 | partitionRecordSize);
75 | return MongoSingleSplitter.split(splitContext);
76 | }
77 |
78 | int numberOfPartitions =
79 | (int) (Math.ceil(totalNumOfDocuments / (double) partitionRecordSize));
80 |
81 | BsonDocument lastUpperBound = null;
82 | List paginatedSplits = new ArrayList<>();
83 |
84 | for (int splitNum = 0; splitNum < numberOfPartitions; splitNum++) {
85 | List pipeline = new ArrayList<>();
86 |
87 | pipeline.add(Aggregates.project(Projections.include(ID_FIELD)));
88 | pipeline.add(Aggregates.project(Sorts.ascending(ID_FIELD)));
89 |
90 | // We don't have to set the upper bounds limit if we're generating the first split.
91 | if (lastUpperBound != null) {
92 | BsonDocument matchFilter = new BsonDocument();
93 | if (lastUpperBound.containsKey(ID_FIELD)) {
94 | matchFilter.put(
95 | ID_FIELD, new BsonDocument("$gte", lastUpperBound.get(ID_FIELD)));
96 | }
97 | pipeline.add(Aggregates.match(matchFilter));
98 | }
99 | pipeline.add(Aggregates.skip(partitionRecordSize));
100 | pipeline.add(Aggregates.limit(1));
101 |
102 | BsonDocument currentUpperBound =
103 | splitContext
104 | .getMongoCollection()
105 | .aggregate(pipeline)
106 | .allowDiskUse(true)
107 | .first();
108 |
109 | paginatedSplits.add(
110 | new MongoScanSourceSplit(
111 | String.format("%s_%d", namespace, splitNum),
112 | namespace.getDatabaseName(),
113 | namespace.getCollectionName(),
114 | lastUpperBound != null ? lastUpperBound : BSON_MIN_BOUNDARY,
115 | currentUpperBound != null ? currentUpperBound : BSON_MAX_BOUNDARY,
116 | ID_HINT));
117 |
118 | if (currentUpperBound == null) {
119 | break;
120 | }
121 | lastUpperBound = currentUpperBound;
122 | }
123 |
124 | return paginatedSplits;
125 | }
126 | }
127 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.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.connector.mongodb.source.enumerator.splitter;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
22 | import org.apache.flink.util.FlinkRuntimeException;
23 |
24 | import com.mongodb.MongoException;
25 | import com.mongodb.MongoNamespace;
26 | import com.mongodb.client.MongoClient;
27 | import org.bson.BsonDocument;
28 | import org.slf4j.Logger;
29 | import org.slf4j.LoggerFactory;
30 |
31 | import java.util.ArrayList;
32 | import java.util.Collection;
33 | import java.util.List;
34 | import java.util.Optional;
35 |
36 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
37 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
38 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
39 | import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
40 | import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
41 | import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
42 |
43 | /**
44 | * Sharded Partitioner
45 | *
46 | *
Uses the chunks collection and partitions the collection based on the sharded collections
47 | * chunk ranges.
48 | *
49 | *
The following config collections' read privilege is required.
50 | *
51 | *
52 | *
config.collections
53 | *
config.chunks
54 | *
55 | */
56 | @Internal
57 | public class MongoShardedSplitter {
58 |
59 | private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
60 |
61 | private MongoShardedSplitter() {}
62 |
63 | public static Collection split(MongoSplitContext splitContext) {
64 | MongoNamespace namespace = splitContext.getMongoNamespace();
65 | MongoClient mongoClient = splitContext.getMongoClient();
66 |
67 | List chunks;
68 | Optional collectionMetadata;
69 | try {
70 | collectionMetadata = readCollectionMetadata(mongoClient, namespace);
71 | if (!collectionMetadata.isPresent()) {
72 | LOG.error(
73 | "Do sharded split failed, collection {} does not appear to be sharded.",
74 | namespace);
75 | throw new FlinkRuntimeException(
76 | String.format(
77 | "Do sharded split failed, %s is not a sharded collection.",
78 | namespace));
79 | }
80 |
81 | if (isShardedCollectionDropped(collectionMetadata.get())) {
82 | LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
83 | throw new FlinkRuntimeException(
84 | String.format("Do sharded split failed, %s was dropped.", namespace));
85 | }
86 |
87 | chunks = readChunks(mongoClient, collectionMetadata.get());
88 | if (chunks.isEmpty()) {
89 | LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
90 | throw new FlinkRuntimeException(
91 | String.format(
92 | "Do sharded split failed, chunks of %s is empty.", namespace));
93 | }
94 | } catch (MongoException e) {
95 | LOG.error(
96 | "Read chunks from {} failed with error message: {}", namespace, e.getMessage());
97 | throw new FlinkRuntimeException(e);
98 | }
99 |
100 | List sourceSplits = new ArrayList<>(chunks.size());
101 | for (int i = 0; i < chunks.size(); i++) {
102 | BsonDocument chunk = chunks.get(i);
103 | sourceSplits.add(
104 | new MongoScanSourceSplit(
105 | String.format("%s_%d", namespace, i),
106 | namespace.getDatabaseName(),
107 | namespace.getCollectionName(),
108 | chunk.getDocument(MIN_FIELD),
109 | chunk.getDocument(MAX_FIELD),
110 | collectionMetadata.get().getDocument(KEY_FIELD)));
111 | }
112 |
113 | return sourceSplits;
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSingleSplitter.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.connector.mongodb.source.enumerator.splitter;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
22 |
23 | import java.util.Collection;
24 |
25 | import static java.util.Collections.singletonList;
26 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_BOUNDARY;
27 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_BOUNDARY;
28 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
29 |
30 | /** Mongo Splitter that splits MongoDB collection as a single split. */
31 | @Internal
32 | public class MongoSingleSplitter {
33 |
34 | private MongoSingleSplitter() {}
35 |
36 | public static Collection split(MongoSplitContext splitContext) {
37 | MongoScanSourceSplit singleSplit =
38 | new MongoScanSourceSplit(
39 | splitContext.getMongoNamespace().getFullName(),
40 | splitContext.getDatabaseName(),
41 | splitContext.getCollectionName(),
42 | BSON_MIN_BOUNDARY,
43 | BSON_MAX_BOUNDARY,
44 | ID_HINT);
45 |
46 | return singletonList(singleSplit);
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitContext.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.connector.mongodb.source.enumerator.splitter;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
23 |
24 | import com.mongodb.MongoNamespace;
25 | import com.mongodb.client.MongoClient;
26 | import com.mongodb.client.MongoCollection;
27 | import org.bson.BsonBoolean;
28 | import org.bson.BsonDocument;
29 | import org.bson.BsonInt64;
30 |
31 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
32 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_FIELD;
33 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARDED_FIELD;
34 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
35 |
36 | /**
37 | * The split context used by {@link MongoSplitters} to split collection into a set of chunks for
38 | * MongoDB data source.
39 | */
40 | @Internal
41 | public class MongoSplitContext {
42 |
43 | /** Read options of MongoDB. */
44 | private final MongoReadOptions readOptions;
45 |
46 | /** Client of MongoDB. */
47 | private final MongoClient mongoClient;
48 |
49 | /** Namespace of MongoDB, eg. db.coll. */
50 | private final MongoNamespace namespace;
51 |
52 | /** Is a sharded collection. */
53 | private final boolean sharded;
54 |
55 | /** The number of objects or documents in this collection. */
56 | private final long count;
57 |
58 | /** The total uncompressed size(bytes) in memory of all records in a collection. */
59 | private final long size;
60 |
61 | /** The average size(bytes) of an object in the collection. */
62 | private final long avgObjSize;
63 |
64 | public MongoSplitContext(
65 | MongoReadOptions readOptions,
66 | MongoClient mongoClient,
67 | MongoNamespace namespace,
68 | boolean sharded,
69 | long count,
70 | long size,
71 | long avgObjSize) {
72 | this.readOptions = readOptions;
73 | this.mongoClient = mongoClient;
74 | this.namespace = namespace;
75 | this.sharded = sharded;
76 | this.count = count;
77 | this.size = size;
78 | this.avgObjSize = avgObjSize;
79 | }
80 |
81 | public static MongoSplitContext of(
82 | MongoReadOptions readOptions,
83 | MongoClient mongoClient,
84 | MongoNamespace namespace,
85 | BsonDocument collStats) {
86 | return new MongoSplitContext(
87 | readOptions,
88 | mongoClient,
89 | namespace,
90 | collStats.getBoolean(SHARDED_FIELD, BsonBoolean.FALSE).getValue(),
91 | collStats.getNumber(COUNT_FIELD, new BsonInt64(0)).longValue(),
92 | collStats.getNumber(SIZE_FIELD, new BsonInt64(0)).longValue(),
93 | collStats.getNumber(AVG_OBJ_SIZE_FIELD, new BsonInt64(0)).longValue());
94 | }
95 |
96 | public MongoClient getMongoClient() {
97 | return mongoClient;
98 | }
99 |
100 | public MongoReadOptions getReadOptions() {
101 | return readOptions;
102 | }
103 |
104 | public String getDatabaseName() {
105 | return namespace.getDatabaseName();
106 | }
107 |
108 | public String getCollectionName() {
109 | return namespace.getCollectionName();
110 | }
111 |
112 | public MongoNamespace getMongoNamespace() {
113 | return namespace;
114 | }
115 |
116 | public MongoCollection getMongoCollection() {
117 | return mongoClient
118 | .getDatabase(namespace.getDatabaseName())
119 | .getCollection(namespace.getCollectionName())
120 | .withDocumentClass(BsonDocument.class);
121 | }
122 |
123 | public boolean isSharded() {
124 | return sharded;
125 | }
126 |
127 | public long getCount() {
128 | return count;
129 | }
130 |
131 | public long getSize() {
132 | return size;
133 | }
134 |
135 | public long getAvgObjSize() {
136 | return avgObjSize;
137 | }
138 | }
139 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.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.connector.mongodb.source.enumerator.splitter;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.configuration.MemorySize;
22 | import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
23 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
24 | import org.apache.flink.util.FlinkRuntimeException;
25 |
26 | import com.mongodb.MongoException;
27 | import com.mongodb.MongoNamespace;
28 | import com.mongodb.client.MongoClient;
29 | import org.apache.commons.collections.CollectionUtils;
30 | import org.bson.BsonArray;
31 | import org.bson.BsonDocument;
32 | import org.bson.BsonInt32;
33 | import org.bson.BsonValue;
34 | import org.slf4j.Logger;
35 | import org.slf4j.LoggerFactory;
36 |
37 | import java.util.ArrayList;
38 | import java.util.Collection;
39 | import java.util.List;
40 |
41 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_BOUNDARY;
42 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
43 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
44 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
45 | import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
46 | import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
47 |
48 | /**
49 | * SplitVector Partitioner
50 | *
51 | *
Uses the SplitVector command to generate chunks for a collection. eg.
52 | * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})
53 | *
54 | *
Requires splitVector privilege.
55 | */
56 | @Internal
57 | public class MongoSplitVectorSplitter {
58 |
59 | private static final Logger LOG = LoggerFactory.getLogger(MongoSplitVectorSplitter.class);
60 |
61 | private MongoSplitVectorSplitter() {}
62 |
63 | public static Collection split(MongoSplitContext splitContext) {
64 | if (splitContext.isSharded()) {
65 | throw new FlinkRuntimeException("splitVector does not apply to sharded collections.");
66 | }
67 |
68 | MongoClient mongoClient = splitContext.getMongoClient();
69 | MongoNamespace namespace = splitContext.getMongoNamespace();
70 | MongoReadOptions readOptions = splitContext.getReadOptions();
71 |
72 | MemorySize chunkSize = readOptions.getPartitionSize();
73 | // if partition size < 1mb, use 1 mb as chunk size.
74 | int maxChunkSizeMB = Math.max(chunkSize.getMebiBytes(), 1);
75 |
76 | BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1));
77 |
78 | BsonDocument splitResult;
79 | try {
80 | splitResult = splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB);
81 | } catch (MongoException e) {
82 | LOG.error("Execute splitVector command failed : {}", e.getMessage());
83 | throw new FlinkRuntimeException(e);
84 | }
85 |
86 | BsonArray splitKeys = splitResult.getArray(SPLIT_KEYS_FIELD);
87 | if (CollectionUtils.isEmpty(splitKeys)) {
88 | // documents size is less than chunk size, treat the entire collection as single chunk.
89 | return MongoSingleSplitter.split(splitContext);
90 | }
91 |
92 | // Complete right bound: (lastKey, maxKey)
93 | splitKeys.add(BSON_MAX_BOUNDARY);
94 |
95 | List sourceSplits = new ArrayList<>(splitKeys.size());
96 |
97 | BsonValue lowerValue = BSON_MIN_KEY;
98 | for (int i = 0; i < splitKeys.size(); i++) {
99 | BsonValue splitKeyValue = splitKeys.get(i).asDocument().get(ID_FIELD);
100 | sourceSplits.add(
101 | new MongoScanSourceSplit(
102 | String.format("%s_%d", namespace, i),
103 | namespace.getDatabaseName(),
104 | namespace.getCollectionName(),
105 | new BsonDocument(ID_FIELD, lowerValue),
106 | new BsonDocument(ID_FIELD, splitKeyValue),
107 | ID_HINT));
108 | lowerValue = splitKeyValue;
109 | }
110 |
111 | return sourceSplits;
112 | }
113 | }
114 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.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.connector.mongodb.source.enumerator.splitter;
20 |
21 | import org.apache.flink.annotation.Internal;
22 | import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
23 | import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
24 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
25 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
26 | import org.apache.flink.util.FlinkRuntimeException;
27 |
28 | import com.mongodb.MongoException;
29 | import com.mongodb.MongoNamespace;
30 | import com.mongodb.client.MongoClient;
31 | import org.bson.BsonDocument;
32 | import org.slf4j.Logger;
33 | import org.slf4j.LoggerFactory;
34 |
35 | import java.util.Collection;
36 |
37 | /** To split collections of MongoDB to {@link MongoSourceSplit}s. */
38 | @Internal
39 | public class MongoSplitters {
40 |
41 | private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
42 |
43 | private MongoSplitters() {}
44 |
45 | public static Collection split(
46 | MongoClient mongoClient, MongoReadOptions readOptions, MongoNamespace namespace) {
47 | BsonDocument collStats;
48 | try {
49 | collStats = MongoUtils.collStats(mongoClient, namespace);
50 | } catch (MongoException e) {
51 | LOG.error("Execute collStats command failed, with error message: {}", e.getMessage());
52 | throw new FlinkRuntimeException(e);
53 | }
54 |
55 | MongoSplitContext splitContext =
56 | MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
57 |
58 | switch (readOptions.getPartitionStrategy()) {
59 | case SINGLE:
60 | return MongoSingleSplitter.split(splitContext);
61 | case SAMPLE:
62 | return MongoSampleSplitter.split(splitContext);
63 | case SPLIT_VECTOR:
64 | return MongoSplitVectorSplitter.split(splitContext);
65 | case SHARDED:
66 | return MongoShardedSplitter.split(splitContext);
67 | case PAGINATION:
68 | return MongoPaginationSplitter.split(splitContext);
69 | case DEFAULT:
70 | default:
71 | return splitContext.isSharded()
72 | ? MongoShardedSplitter.split(splitContext)
73 | : MongoSplitVectorSplitter.split(splitContext);
74 | }
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/PartitionStrategy.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.connector.mongodb.source.enumerator.splitter;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.configuration.DescribedEnum;
22 | import org.apache.flink.configuration.description.InlineElement;
23 |
24 | import static org.apache.flink.configuration.description.TextElement.text;
25 |
26 | /**
27 | * Partition strategies that can be chosen. Available strategies are single, sample, split-vector,
28 | * sharded and default.
29 | *
30 | *
31 | *
single: treats the entire collection as a single partition.
32 | *
sample: samples the collection and generate partitions which is fast but possibly uneven.
33 | *
split-vector: uses the splitVector command to generate partitions for non-sharded
34 | * collections which is fast and even. The splitVector permission is required.
35 | *
sharded: reads config.chunks (MongoDB splits a sharded collection into chunks, and the
36 | * range of the chunks are stored within the collection) as the partitions directly. The
37 | * sharded strategy only used for sharded collection which is fast and even. Read permission
38 | * of config database is required.
39 | *
default: uses sharded strategy for sharded collections otherwise using split vector
40 | * strategy.
41 | *
42 | */
43 | @PublicEvolving
44 | public enum PartitionStrategy implements DescribedEnum {
45 | SINGLE("single", text("Do not split, treat a collection as a single chunk.")),
46 |
47 | SAMPLE("sample", text("Randomly sample the collection, then splits to multiple chunks.")),
48 |
49 | SPLIT_VECTOR(
50 | "split-vector",
51 | text("Uses the SplitVector command to generate chunks for non-sharded collections.")),
52 |
53 | SHARDED(
54 | "sharded",
55 | text(
56 | "Read the chunk ranges from config.chunks collection and splits to multiple chunks. Only support sharded collections.")),
57 |
58 | PAGINATION(
59 | "pagination",
60 | text(
61 | "Creating chunk records evenly by count. Each chunk will have exactly the same number of records.")),
62 |
63 | DEFAULT(
64 | "default",
65 | text(
66 | "Using sharded strategy for sharded collections"
67 | + " otherwise using split vector strategy."));
68 |
69 | private final String name;
70 | private final InlineElement description;
71 |
72 | PartitionStrategy(String name, InlineElement description) {
73 | this.name = name;
74 | this.description = description;
75 | }
76 |
77 | @Override
78 | public InlineElement getDescription() {
79 | return description;
80 | }
81 |
82 | @Override
83 | public String toString() {
84 | return name;
85 | }
86 | }
87 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.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.connector.mongodb.source.reader;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.base.source.reader.RecordEmitter;
22 | import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
23 | import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
24 | import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
25 | import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
26 | import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
27 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
28 | import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplitState;
29 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
30 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitState;
31 |
32 | import org.bson.BsonDocument;
33 | import org.slf4j.Logger;
34 | import org.slf4j.LoggerFactory;
35 |
36 | import java.util.Map;
37 | import java.util.function.Supplier;
38 |
39 | /**
40 | * The common mongo source reader for both ordered & unordered message consuming.
41 | *
42 | * @param The output message type for flink.
43 | */
44 | @Internal
45 | public class MongoSourceReader
46 | extends SingleThreadMultiplexSourceReaderBase<
47 | BsonDocument, OUT, MongoSourceSplit, MongoSourceSplitState> {
48 |
49 | private static final Logger LOG = LoggerFactory.getLogger(MongoSourceReader.class);
50 |
51 | public MongoSourceReader(
52 | FutureCompletingBlockingQueue> elementQueue,
53 | Supplier> splitReaderSupplier,
54 | RecordEmitter recordEmitter,
55 | MongoSourceReaderContext readerContext) {
56 | super(
57 | elementQueue,
58 | new SingleThreadFetcherManager<>(elementQueue, splitReaderSupplier),
59 | recordEmitter,
60 | readerContext.getConfiguration(),
61 | readerContext);
62 | }
63 |
64 | @Override
65 | public void start() {
66 | if (getNumberOfCurrentlyAssignedSplits() == 0) {
67 | context.sendSplitRequest();
68 | }
69 | }
70 |
71 | @Override
72 | protected void onSplitFinished(Map finishedSplitIds) {
73 | for (MongoSourceSplitState splitState : finishedSplitIds.values()) {
74 | MongoSourceSplit sourceSplit = splitState.toMongoSourceSplit();
75 | LOG.info("Split {} is finished.", sourceSplit.splitId());
76 | }
77 | context.sendSplitRequest();
78 | }
79 |
80 | @Override
81 | protected MongoSourceSplitState initializedState(MongoSourceSplit split) {
82 | if (split instanceof MongoScanSourceSplit) {
83 | return new MongoScanSourceSplitState((MongoScanSourceSplit) split);
84 | } else {
85 | throw new IllegalArgumentException("Unknown split type.");
86 | }
87 | }
88 |
89 | @Override
90 | protected MongoSourceSplit toSplitType(String splitId, MongoSourceSplitState splitState) {
91 | return splitState.toMongoSourceSplit();
92 | }
93 | }
94 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReaderContext.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 | * http://www.apache.org/licenses/LICENSE-2.0
10 | * Unless required by applicable law or agreed to in writing,
11 | * software distributed under the License is distributed on an
12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
13 | * KIND, either express or implied. See the License for the
14 | * specific language governing permissions and limitations
15 | * under the License.
16 | */
17 |
18 | package org.apache.flink.connector.mongodb.source.reader;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.api.connector.source.SourceEvent;
22 | import org.apache.flink.api.connector.source.SourceReaderContext;
23 | import org.apache.flink.configuration.Configuration;
24 | import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader;
25 | import org.apache.flink.metrics.groups.SourceReaderMetricGroup;
26 | import org.apache.flink.util.UserCodeClassLoader;
27 |
28 | import java.util.concurrent.atomic.AtomicInteger;
29 |
30 | /**
31 | * A wrapper class that wraps {@link SourceReaderContext} for sharing message between {@link
32 | * MongoSourceReader} and {@link MongoScanSourceSplitReader}.
33 | */
34 | @Internal
35 | public class MongoSourceReaderContext implements SourceReaderContext {
36 |
37 | private final SourceReaderContext readerContext;
38 | private final AtomicInteger readCount = new AtomicInteger(0);
39 | private final int limit;
40 |
41 | public MongoSourceReaderContext(SourceReaderContext readerContext, int limit) {
42 | this.readerContext = readerContext;
43 | this.limit = limit;
44 | }
45 |
46 | @Override
47 | public SourceReaderMetricGroup metricGroup() {
48 | return readerContext.metricGroup();
49 | }
50 |
51 | @Override
52 | public Configuration getConfiguration() {
53 | return readerContext.getConfiguration();
54 | }
55 |
56 | @Override
57 | public String getLocalHostName() {
58 | return readerContext.getLocalHostName();
59 | }
60 |
61 | @Override
62 | public int getIndexOfSubtask() {
63 | return readerContext.getIndexOfSubtask();
64 | }
65 |
66 | @Override
67 | public void sendSplitRequest() {
68 | readerContext.sendSplitRequest();
69 | }
70 |
71 | @Override
72 | public void sendSourceEventToCoordinator(SourceEvent sourceEvent) {
73 | readerContext.sendSourceEventToCoordinator(sourceEvent);
74 | }
75 |
76 | @Override
77 | public UserCodeClassLoader getUserCodeClassLoader() {
78 | return readerContext.getUserCodeClassLoader();
79 | }
80 |
81 | public AtomicInteger getReadCount() {
82 | return readCount;
83 | }
84 |
85 | public boolean isLimitPushedDown() {
86 | return limit > 0;
87 | }
88 |
89 | public boolean isOverLimit() {
90 | return limit > 0 && readCount.get() >= limit;
91 | }
92 |
93 | public int getLimit() {
94 | return limit;
95 | }
96 | }
97 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.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.connector.mongodb.source.reader.deserializer;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
22 | import org.apache.flink.util.Collector;
23 |
24 | import org.bson.BsonDocument;
25 |
26 | import java.io.IOException;
27 | import java.io.Serializable;
28 |
29 | /**
30 | * A schema bridge for deserializing the MongoDB's {@code BsonDocument} into a flink managed
31 | * instance.
32 | *
33 | * @param The output message type for sinking to downstream flink operator.
34 | */
35 | @PublicEvolving
36 | public interface MongoDeserializationSchema extends Serializable, ResultTypeQueryable {
37 |
38 | /**
39 | * Deserializes the BSON document.
40 | *
41 | * @param document The BSON document to deserialize.
42 | * @return The deserialized message as an object (null if the message cannot be deserialized).
43 | */
44 | T deserialize(BsonDocument document) throws IOException;
45 |
46 | /**
47 | * Deserializes the BSON document.
48 | *
49 | *
Can output multiple records through the {@link Collector}. Note that number and size of
50 | * the produced records should be relatively small. Depending on the source implementation
51 | * records can be buffered in memory or collecting records might delay emitting checkpoint
52 | * barrier.
53 | *
54 | * @param document The BSON document to deserialize.
55 | * @param out The collector to put the resulting messages.
56 | */
57 | default void deserialize(BsonDocument document, Collector out) throws IOException {
58 | T deserialize = deserialize(document);
59 | if (deserialize != null) {
60 | out.collect(deserialize);
61 | }
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/emitter/MongoRecordEmitter.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.connector.mongodb.source.reader.emitter;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.api.connector.source.SourceOutput;
22 | import org.apache.flink.connector.base.source.reader.RecordEmitter;
23 | import org.apache.flink.connector.mongodb.source.reader.MongoSourceReader;
24 | import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema;
25 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitState;
26 | import org.apache.flink.util.Collector;
27 |
28 | import org.bson.BsonDocument;
29 |
30 | /**
31 | * The {@link RecordEmitter} implementation for {@link MongoSourceReader} . We would always update
32 | * the last consumed message id in this emitter.
33 | */
34 | @Internal
35 | public class MongoRecordEmitter
36 | implements RecordEmitter {
37 |
38 | private final MongoDeserializationSchema deserializationSchema;
39 | private final SourceOutputWrapper sourceOutputWrapper;
40 |
41 | public MongoRecordEmitter(MongoDeserializationSchema deserializationSchema) {
42 | this.deserializationSchema = deserializationSchema;
43 | this.sourceOutputWrapper = new SourceOutputWrapper<>();
44 | }
45 |
46 | @Override
47 | public void emitRecord(
48 | BsonDocument document, SourceOutput output, MongoSourceSplitState splitState)
49 | throws Exception {
50 | // Update current offset.
51 | splitState.updateOffset(document);
52 | // Sink the record to source output.
53 | sourceOutputWrapper.setSourceOutput(output);
54 | deserializationSchema.deserialize(document, sourceOutputWrapper);
55 | }
56 |
57 | private static class SourceOutputWrapper implements Collector {
58 | private SourceOutput sourceOutput;
59 |
60 | @Override
61 | public void collect(T record) {
62 | sourceOutput.collect(record);
63 | }
64 |
65 | @Override
66 | public void close() {}
67 |
68 | private void setSourceOutput(SourceOutput sourceOutput) {
69 | this.sourceOutput = sourceOutput;
70 | }
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.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.connector.mongodb.source.reader.split;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
22 | import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
23 |
24 | import org.bson.BsonDocument;
25 |
26 | /**
27 | * A split reader implements {@link SplitReader} for {@link MongoSourceSplit}.
28 | *
29 | * @param Mongo source split.
30 | */
31 | @Internal
32 | public interface MongoSourceSplitReader
33 | extends SplitReader {}
34 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplit.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.connector.mongodb.source.split;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.api.connector.source.SourceSplit;
22 |
23 | import org.bson.BsonDocument;
24 |
25 | import java.util.Objects;
26 |
27 | /** A {@link SourceSplit} implementation for a MongoDB's partition. */
28 | @PublicEvolving
29 | public class MongoScanSourceSplit extends MongoSourceSplit {
30 |
31 | private static final long serialVersionUID = 1L;
32 |
33 | private final String database;
34 |
35 | private final String collection;
36 |
37 | private final BsonDocument min;
38 |
39 | private final BsonDocument max;
40 |
41 | private final BsonDocument hint;
42 |
43 | private final int offset;
44 |
45 | public MongoScanSourceSplit(
46 | String splitId,
47 | String database,
48 | String collection,
49 | BsonDocument min,
50 | BsonDocument max,
51 | BsonDocument hint) {
52 | this(splitId, database, collection, min, max, hint, 0);
53 | }
54 |
55 | public MongoScanSourceSplit(
56 | String splitId,
57 | String database,
58 | String collection,
59 | BsonDocument min,
60 | BsonDocument max,
61 | BsonDocument hint,
62 | int offset) {
63 | super(splitId);
64 | this.database = database;
65 | this.collection = collection;
66 | this.min = min;
67 | this.max = max;
68 | this.hint = hint;
69 | this.offset = offset;
70 | }
71 |
72 | public String getDatabase() {
73 | return database;
74 | }
75 |
76 | public String getCollection() {
77 | return collection;
78 | }
79 |
80 | public BsonDocument getMin() {
81 | return min;
82 | }
83 |
84 | public BsonDocument getMax() {
85 | return max;
86 | }
87 |
88 | public BsonDocument getHint() {
89 | return hint;
90 | }
91 |
92 | public int getOffset() {
93 | return offset;
94 | }
95 |
96 | @Override
97 | public boolean equals(Object o) {
98 | if (this == o) {
99 | return true;
100 | }
101 | if (o == null || getClass() != o.getClass()) {
102 | return false;
103 | }
104 | if (!super.equals(o)) {
105 | return false;
106 | }
107 | MongoScanSourceSplit split = (MongoScanSourceSplit) o;
108 | return Objects.equals(database, split.database)
109 | && Objects.equals(collection, split.collection)
110 | && Objects.equals(min, split.min)
111 | && Objects.equals(max, split.max)
112 | && Objects.equals(hint, split.hint)
113 | && offset == split.offset;
114 | }
115 |
116 | @Override
117 | public int hashCode() {
118 | return Objects.hash(super.hashCode(), database, collection, min, max, hint, offset);
119 | }
120 |
121 | @Override
122 | public String toString() {
123 | return "MongoScanSourceSplit {"
124 | + " splitId="
125 | + splitId
126 | + ", database="
127 | + database
128 | + ", collection="
129 | + collection
130 | + ", min="
131 | + min
132 | + ", max="
133 | + max
134 | + ", hint="
135 | + hint
136 | + ", offset="
137 | + offset
138 | + " }";
139 | }
140 | }
141 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplitState.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 | * http://www.apache.org/licenses/LICENSE-2.0
10 | * Unless required by applicable law or agreed to in writing,
11 | * software distributed under the License is distributed on an
12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
13 | * KIND, either express or implied. See the License for the
14 | * specific language governing permissions and limitations
15 | * under the License.
16 | */
17 |
18 | package org.apache.flink.connector.mongodb.source.split;
19 |
20 | import org.apache.flink.annotation.Internal;
21 |
22 | import org.bson.BsonDocument;
23 |
24 | /** MongoDB source split state for {@link MongoScanSourceSplit}. */
25 | @Internal
26 | public class MongoScanSourceSplitState implements MongoSourceSplitState {
27 |
28 | private final MongoScanSourceSplit scanSplit;
29 |
30 | private int offset;
31 |
32 | public MongoScanSourceSplitState(MongoScanSourceSplit scanSplit) {
33 | this.scanSplit = scanSplit;
34 | this.offset = scanSplit.getOffset();
35 | }
36 |
37 | @Override
38 | public MongoScanSourceSplit toMongoSourceSplit() {
39 | return new MongoScanSourceSplit(
40 | scanSplit.splitId(),
41 | scanSplit.getDatabase(),
42 | scanSplit.getCollection(),
43 | scanSplit.getMin(),
44 | scanSplit.getMax(),
45 | scanSplit.getHint(),
46 | offset);
47 | }
48 |
49 | @Override
50 | public void updateOffset(BsonDocument record) {
51 | offset++;
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplit.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.connector.mongodb.source.split;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.api.connector.source.SourceSplit;
22 |
23 | import java.io.Serializable;
24 | import java.util.Objects;
25 |
26 | /** A super class of {@link SourceSplit} implementation for a MongoDB's source split. */
27 | @PublicEvolving
28 | public abstract class MongoSourceSplit implements SourceSplit, Serializable {
29 |
30 | protected final String splitId;
31 |
32 | protected MongoSourceSplit(String splitId) {
33 | this.splitId = splitId;
34 | }
35 |
36 | @Override
37 | public String splitId() {
38 | return splitId;
39 | }
40 |
41 | @Override
42 | public boolean equals(Object o) {
43 | if (this == o) {
44 | return true;
45 | }
46 | if (o == null || getClass() != o.getClass()) {
47 | return false;
48 | }
49 | MongoSourceSplit that = (MongoSourceSplit) o;
50 | return Objects.equals(splitId, that.splitId);
51 | }
52 |
53 | @Override
54 | public int hashCode() {
55 | return Objects.hash(splitId);
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializer.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.connector.mongodb.source.split;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.core.io.SimpleVersionedSerializer;
22 |
23 | import org.bson.BsonDocument;
24 |
25 | import java.io.ByteArrayInputStream;
26 | import java.io.ByteArrayOutputStream;
27 | import java.io.DataInputStream;
28 | import java.io.DataOutputStream;
29 | import java.io.IOException;
30 |
31 | /** The {@link SimpleVersionedSerializer serializer} for {@link MongoSourceSplit}. */
32 | @Internal
33 | public class MongoSourceSplitSerializer implements SimpleVersionedSerializer {
34 |
35 | public static final MongoSourceSplitSerializer INSTANCE = new MongoSourceSplitSerializer();
36 |
37 | // This version should be bumped after modifying the MongoSourceSplit.
38 | public static final int CURRENT_VERSION = 0;
39 |
40 | public static final int SCAN_SPLIT_FLAG = 1;
41 |
42 | private MongoSourceSplitSerializer() {}
43 |
44 | @Override
45 | public int getVersion() {
46 | return CURRENT_VERSION;
47 | }
48 |
49 | @Override
50 | public byte[] serialize(MongoSourceSplit obj) throws IOException {
51 | // VERSION 0 serialization
52 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
53 | DataOutputStream out = new DataOutputStream(baos)) {
54 | serializeMongoSplit(out, obj);
55 | out.flush();
56 | return baos.toByteArray();
57 | }
58 | }
59 |
60 | @Override
61 | public MongoSourceSplit deserialize(int version, byte[] serialized) throws IOException {
62 | // VERSION 0 deserialization
63 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
64 | DataInputStream in = new DataInputStream(bais)) {
65 | int splitKind = in.readInt();
66 | if (splitKind == SCAN_SPLIT_FLAG) {
67 | return deserializeMongoScanSourceSplit(version, in);
68 | }
69 | throw new IOException("Unknown split kind: " + splitKind);
70 | }
71 | }
72 |
73 | public void serializeMongoSplit(DataOutputStream out, MongoSourceSplit obj) throws IOException {
74 | if (obj instanceof MongoScanSourceSplit) {
75 | MongoScanSourceSplit split = (MongoScanSourceSplit) obj;
76 | out.writeInt(SCAN_SPLIT_FLAG);
77 | out.writeUTF(split.splitId());
78 | out.writeUTF(split.getDatabase());
79 | out.writeUTF(split.getCollection());
80 | out.writeUTF(split.getMin().toJson());
81 | out.writeUTF(split.getMax().toJson());
82 | out.writeUTF(split.getHint().toJson());
83 | out.writeInt(split.getOffset());
84 | }
85 | }
86 |
87 | public MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)
88 | throws IOException {
89 | switch (version) {
90 | case 0:
91 | String splitId = in.readUTF();
92 | String database = in.readUTF();
93 | String collection = in.readUTF();
94 | BsonDocument min = BsonDocument.parse(in.readUTF());
95 | BsonDocument max = BsonDocument.parse(in.readUTF());
96 | BsonDocument hint = BsonDocument.parse(in.readUTF());
97 | int offset = in.readInt();
98 | return new MongoScanSourceSplit(
99 | splitId, database, collection, min, max, hint, offset);
100 | default:
101 | throw new IOException("Unknown version: " + version);
102 | }
103 | }
104 | }
105 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.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.connector.mongodb.source.split;
19 |
20 | import org.apache.flink.annotation.Internal;
21 |
22 | import org.bson.BsonDocument;
23 |
24 | /** MongoDB source split state for {@link MongoSourceSplit}. */
25 | @Internal
26 | public interface MongoSourceSplitState {
27 |
28 | /** Use the current split state to create a new {@link MongoSourceSplit}. */
29 | MongoSourceSplit toMongoSourceSplit();
30 |
31 | /**
32 | * Update the offset read by the current split for failure recovery.
33 | *
34 | * @param record The latest record that was read.
35 | */
36 | void updateOffset(BsonDocument record);
37 | }
38 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/FilterHandlingPolicy.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.connector.mongodb.table;
19 |
20 | import org.apache.flink.annotation.PublicEvolving;
21 | import org.apache.flink.configuration.DescribedEnum;
22 | import org.apache.flink.configuration.description.InlineElement;
23 |
24 | import static org.apache.flink.configuration.description.TextElement.text;
25 |
26 | /** Fine-grained configuration to control filter push down for MongoDB Table/SQL source. */
27 | @PublicEvolving
28 | public enum FilterHandlingPolicy implements DescribedEnum {
29 | ALWAYS("always", text("Always push the supported filters to MongoDB.")),
30 |
31 | NEVER("never", text("Never push any filters to MongoDB."));
32 |
33 | private final String name;
34 | private final InlineElement description;
35 |
36 | FilterHandlingPolicy(String name, InlineElement description) {
37 | this.name = name;
38 | this.description = description;
39 | }
40 |
41 | @Override
42 | public InlineElement getDescription() {
43 | return description;
44 | }
45 |
46 | @Override
47 | public String toString() {
48 | return name;
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoShardKeysExtractor.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 | * http://www.apache.org/licenses/LICENSE-2.0
10 | * Unless required by applicable law or agreed to in writing,
11 | * software distributed under the License is distributed on an
12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
13 | * KIND, either express or implied. See the License for the
14 | * specific language governing permissions and limitations
15 | * under the License.
16 | */
17 |
18 | package org.apache.flink.connector.mongodb.table;
19 |
20 | import org.apache.flink.annotation.Internal;
21 | import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
22 | import org.apache.flink.table.catalog.ResolvedSchema;
23 | import org.apache.flink.table.connector.Projection;
24 | import org.apache.flink.table.data.RowData;
25 | import org.apache.flink.table.data.utils.ProjectedRowData;
26 | import org.apache.flink.table.types.DataType;
27 | import org.apache.flink.table.types.logical.LogicalType;
28 | import org.apache.flink.util.function.SerializableFunction;
29 |
30 | import org.bson.BsonDocument;
31 | import org.bson.BsonObjectId;
32 | import org.bson.BsonValue;
33 | import org.bson.types.ObjectId;
34 | import org.slf4j.Logger;
35 | import org.slf4j.LoggerFactory;
36 |
37 | import java.util.Arrays;
38 | import java.util.List;
39 | import java.util.Optional;
40 |
41 | /** An extractor for a MongoDB shard keys from a {@link RowData}. */
42 | @Internal
43 | public class MongoShardKeysExtractor implements SerializableFunction {
44 |
45 | private static final long serialVersionUID = 1L;
46 |
47 | private static final Logger LOG = LoggerFactory.getLogger(MongoShardKeysExtractor.class);
48 |
49 | private static final BsonDocument EMPTY_DOCUMENT = new BsonDocument();
50 |
51 | private final SerializableFunction