[mongodb] Match multiple database and collection names using a regular expression in MongoDB. (#940)
Co-authored-by: Bobby Richard <bobbyrlg@gmail.com>pull/947/head
parent
580391b6ee
commit
473e36b9cd
@ -1,48 +0,0 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
// In production you would almost certainly limit the replication user must be on the follower (slave) machine,
|
||||
// to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'.
|
||||
// However, in this database we'll grant 2 users different privileges:
|
||||
//
|
||||
// 1) 'flinkuser' - all privileges required by the snapshot reader AND oplog reader (used for testing)
|
||||
// 2) 'superuser' - all privileges
|
||||
//
|
||||
|
||||
//use admin;
|
||||
if (db.system.users.find({user:'superuser'}).count() == 0) {
|
||||
db.createUser(
|
||||
{
|
||||
user: 'superuser',
|
||||
pwd: 'superpw',
|
||||
roles: [ { role: 'root', db: 'admin' } ]
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
if (db.system.users.find({user:'flinkuser'}).count() == 0) {
|
||||
db.createUser(
|
||||
{
|
||||
user: 'flinkuser',
|
||||
pwd: 'a1?~!@#$%^&*(){}[]<>.,+_-=/|:;',
|
||||
roles: [
|
||||
{ role: 'read', db: 'admin' },
|
||||
{ role: 'readAnyDatabase', db: 'admin' }
|
||||
]
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
rs.status()
|
@ -0,0 +1,190 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.ververica.cdc.connectors.mongodb.utils;
|
||||
|
||||
import com.mongodb.MongoNamespace;
|
||||
import com.mongodb.client.MongoClient;
|
||||
import com.mongodb.client.MongoDatabase;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.bson.BsonDocument;
|
||||
import org.bson.conversions.Bson;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/** Utilities to discovery matched collections. */
|
||||
public class CollectionDiscoveryUtils {
|
||||
|
||||
public static final String REGEX_META_CHARACTERS = ".$|()[]{}<>^?*+-=!\\";
|
||||
|
||||
public static final String ADD_NS_FIELD_NAME = "_ns_";
|
||||
|
||||
public static final Bson ADD_NS_FIELD =
|
||||
BsonDocument.parse(
|
||||
String.format(
|
||||
"{'$addFields': {'%s': {'$concat': ['$ns.db', '.', '$ns.coll']}}}",
|
||||
ADD_NS_FIELD_NAME));
|
||||
|
||||
private CollectionDiscoveryUtils() {}
|
||||
|
||||
public static List<String> databaseNames(
|
||||
MongoClient mongoClient, Predicate<String> databaseFilter) {
|
||||
List<String> databaseNames = new ArrayList<>();
|
||||
mongoClient
|
||||
.listDatabaseNames()
|
||||
.forEach(
|
||||
dbName -> {
|
||||
if (databaseFilter.test(dbName)) {
|
||||
databaseNames.add(dbName);
|
||||
}
|
||||
});
|
||||
return databaseNames;
|
||||
}
|
||||
|
||||
public static List<String> collectionNames(
|
||||
MongoClient mongoClient,
|
||||
List<String> databaseNames,
|
||||
Predicate<String> collectionFilter) {
|
||||
List<String> collectionNames = new ArrayList<>();
|
||||
for (String dbName : databaseNames) {
|
||||
MongoDatabase db = mongoClient.getDatabase(dbName);
|
||||
db.listCollectionNames()
|
||||
.map(collName -> dbName + "." + collName)
|
||||
.forEach(
|
||||
fullName -> {
|
||||
if (collectionFilter.test(fullName)) {
|
||||
collectionNames.add(fullName);
|
||||
}
|
||||
});
|
||||
}
|
||||
return collectionNames;
|
||||
}
|
||||
|
||||
public static Predicate<String> databaseFilter(List<String> databaseList) {
|
||||
Predicate<String> databaseFilter = CollectionDiscoveryUtils::isNotBuiltInDatabase;
|
||||
if (databaseList != null && !databaseList.isEmpty()) {
|
||||
List<Pattern> databasePatterns = includeListAsPatterns(databaseList);
|
||||
databaseFilter = databaseFilter.and(anyMatch(databasePatterns));
|
||||
}
|
||||
return databaseFilter;
|
||||
}
|
||||
|
||||
public static Predicate<String> collectionsFilter(List<String> collectionList) {
|
||||
Predicate<String> collectionFilter = CollectionDiscoveryUtils::isNotBuiltInCollections;
|
||||
if (collectionList != null && !collectionList.isEmpty()) {
|
||||
List<Pattern> collectionPatterns = includeListAsPatterns(collectionList);
|
||||
collectionFilter = collectionFilter.and(anyMatch(collectionPatterns));
|
||||
}
|
||||
return collectionFilter;
|
||||
}
|
||||
|
||||
public static Predicate<String> anyMatch(List<Pattern> patterns) {
|
||||
return s -> {
|
||||
for (Pattern p : patterns) {
|
||||
if (p.matcher(s).matches()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
};
|
||||
}
|
||||
|
||||
public static List<Pattern> includeListAsPatterns(List<String> includeList) {
|
||||
if (includeList != null && !includeList.isEmpty()) {
|
||||
return includeList.stream()
|
||||
// Notice that MongoDB's database and collection names are case-sensitive.
|
||||
// Please refer to https://docs.mongodb.com/manual/reference/limits/
|
||||
// We use case-sensitive pattern here to avoid unexpected results.
|
||||
.map(CollectionDiscoveryUtils::completionPattern)
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isIncludeListExplicitlySpecified(
|
||||
List<String> includeList, List<String> discoveredList) {
|
||||
if (includeList == null || includeList.size() != 1) {
|
||||
return false;
|
||||
}
|
||||
if (discoveredList == null || discoveredList.size() != 1) {
|
||||
return false;
|
||||
}
|
||||
String firstOfIncludeList = includeList.get(0);
|
||||
String firstOfDiscoveredList = discoveredList.get(0);
|
||||
return firstOfDiscoveredList.equals(firstOfIncludeList);
|
||||
}
|
||||
|
||||
public static boolean isNotBuiltInCollections(String fullName) {
|
||||
if (fullName == null) {
|
||||
return false;
|
||||
}
|
||||
MongoNamespace namespace = new MongoNamespace(fullName);
|
||||
return isNotBuiltInDatabase(namespace.getDatabaseName())
|
||||
&& !namespace.getCollectionName().startsWith("system.");
|
||||
}
|
||||
|
||||
public static boolean isNotBuiltInDatabase(String databaseName) {
|
||||
if (databaseName == null) {
|
||||
return false;
|
||||
}
|
||||
return !"local".equals(databaseName)
|
||||
&& !"admin".equals(databaseName)
|
||||
&& !"config".equals(databaseName);
|
||||
}
|
||||
|
||||
public static boolean containsRegexMetaCharacters(String literal) {
|
||||
if (StringUtils.isEmpty(literal)) {
|
||||
return false;
|
||||
}
|
||||
for (int i = 0; i < literal.length(); i++) {
|
||||
if (REGEX_META_CHARACTERS.indexOf(literal.charAt(i)) != -1) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static Pattern completionPattern(String pattern) {
|
||||
if (pattern.startsWith("^") && pattern.endsWith("$")) {
|
||||
return Pattern.compile(pattern);
|
||||
}
|
||||
return Pattern.compile("^(" + pattern + ")$");
|
||||
}
|
||||
|
||||
public static String bsonListToJson(List<Bson> bsonList) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("[");
|
||||
boolean first = true;
|
||||
for (Bson bson : bsonList) {
|
||||
if (first) {
|
||||
first = false;
|
||||
} else {
|
||||
sb.append(",");
|
||||
}
|
||||
sb.append(bson.toBsonDocument().toJson());
|
||||
}
|
||||
sb.append("]");
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
@ -0,0 +1,282 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.ververica.cdc.connectors.mongodb.table;
|
||||
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.TableResult;
|
||||
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
|
||||
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
|
||||
|
||||
import com.mongodb.client.MongoDatabase;
|
||||
import com.ververica.cdc.connectors.mongodb.MongoDBTestBase;
|
||||
import org.bson.Document;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static com.ververica.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSinkSize;
|
||||
import static com.ververica.cdc.connectors.mongodb.utils.MongoDBTestUtils.waitForSnapshotStarted;
|
||||
import static org.hamcrest.Matchers.containsInAnyOrder;
|
||||
import static org.junit.Assert.assertThat;
|
||||
|
||||
/** Integration tests to check mongodb-cdc works well under namespace.regex. */
|
||||
public class MongoDBRegexFilterITCase extends MongoDBTestBase {
|
||||
|
||||
private final StreamExecutionEnvironment env =
|
||||
StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
private final StreamTableEnvironment tEnv =
|
||||
StreamTableEnvironment.create(
|
||||
env,
|
||||
EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build());
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
TestValuesTableFactory.clearAllData();
|
||||
env.setParallelism(1);
|
||||
}
|
||||
|
||||
/** match multiple databases and collections: collection = ^(db0|db1)\.coll_a\d?$ . */
|
||||
@Test
|
||||
public void testMatchMultipleDatabasesAndCollections() throws Exception {
|
||||
// 1. Given collections:
|
||||
// db0: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db0 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
// db1: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db1 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
|
||||
// 2. Test match: collection = ^(db0|db1)\.coll_a\d?$
|
||||
String collectionRegex = String.format("^(%s|%s)\\.coll_a\\d?$", db0, db1);
|
||||
TableResult result = submitTestCase(null, collectionRegex);
|
||||
|
||||
// 3. Wait snapshot finished
|
||||
waitForSinkSize("mongodb_sink", 4);
|
||||
|
||||
// 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
|
||||
// coll_b1.B102]
|
||||
insertRecordsInDatabase(db0);
|
||||
insertRecordsInDatabase(db1);
|
||||
|
||||
// 5. Wait change stream records come
|
||||
waitForSinkSize("mongodb_sink", 8);
|
||||
|
||||
// 6. Check results
|
||||
String[] expected =
|
||||
new String[] {
|
||||
String.format("+I[%s, coll_a1, A101]", db0),
|
||||
String.format("+I[%s, coll_a2, A201]", db0),
|
||||
String.format("+I[%s, coll_a1, A101]", db1),
|
||||
String.format("+I[%s, coll_a2, A201]", db1),
|
||||
String.format("+I[%s, coll_a1, A102]", db0),
|
||||
String.format("+I[%s, coll_a2, A202]", db0),
|
||||
String.format("+I[%s, coll_a1, A102]", db1),
|
||||
String.format("+I[%s, coll_a2, A202]", db1)
|
||||
};
|
||||
|
||||
List<String> actual = TestValuesTableFactory.getResults("mongodb_sink");
|
||||
assertThat(actual, containsInAnyOrder(expected));
|
||||
|
||||
result.getJobClient().get().cancel().get();
|
||||
}
|
||||
|
||||
/** match multiple databases: database = db0|db1 . */
|
||||
@Test
|
||||
public void testMatchMultipleDatabases() throws Exception {
|
||||
// 1. Given collections:
|
||||
// db0: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db0 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
// db1: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db1 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
// db2: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db2 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
|
||||
// 2. Test match database: ^(db0|db1)$
|
||||
String databaseRegex = String.format("%s|%s", db0, db1);
|
||||
TableResult result = submitTestCase(databaseRegex, null);
|
||||
|
||||
// 3. Wait snapshot finished
|
||||
waitForSinkSize("mongodb_sink", 8);
|
||||
|
||||
// 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
|
||||
// coll_b1.B102]
|
||||
insertRecordsInDatabase(db0);
|
||||
insertRecordsInDatabase(db1);
|
||||
insertRecordsInDatabase(db2);
|
||||
|
||||
// 5. Wait change stream records come
|
||||
waitForSinkSize("mongodb_sink", 16);
|
||||
|
||||
// 6. Check results
|
||||
String[] expected =
|
||||
new String[] {
|
||||
String.format("+I[%s, coll_a1, A101]", db0),
|
||||
String.format("+I[%s, coll_a2, A201]", db0),
|
||||
String.format("+I[%s, coll_b1, B101]", db0),
|
||||
String.format("+I[%s, coll_b2, B201]", db0),
|
||||
String.format("+I[%s, coll_a1, A101]", db1),
|
||||
String.format("+I[%s, coll_a2, A201]", db1),
|
||||
String.format("+I[%s, coll_b1, B101]", db1),
|
||||
String.format("+I[%s, coll_b2, B201]", db1),
|
||||
String.format("+I[%s, coll_a1, A102]", db0),
|
||||
String.format("+I[%s, coll_a2, A202]", db0),
|
||||
String.format("+I[%s, coll_b1, B102]", db0),
|
||||
String.format("+I[%s, coll_b2, B202]", db0),
|
||||
String.format("+I[%s, coll_a1, A102]", db1),
|
||||
String.format("+I[%s, coll_a2, A202]", db1),
|
||||
String.format("+I[%s, coll_b1, B102]", db1),
|
||||
String.format("+I[%s, coll_b2, B202]", db1),
|
||||
};
|
||||
|
||||
List<String> actual = TestValuesTableFactory.getResults("mongodb_sink");
|
||||
assertThat(actual, containsInAnyOrder(expected));
|
||||
|
||||
result.getJobClient().get().cancel().get();
|
||||
}
|
||||
|
||||
/** match single database and multiple collections: collection = ^db0\.coll_b\d?$ . */
|
||||
@Test
|
||||
public void testMatchSingleQualifiedCollectionPattern() throws Exception {
|
||||
// 1. Given collections:
|
||||
// db0: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db0 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
// db1: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db1 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
|
||||
// 2. Test match: collection ^(db0|db1)\.coll_a\d?$
|
||||
String collectionRegex = String.format("^%s\\.coll_b\\d?$", db0);
|
||||
TableResult result = submitTestCase(null, collectionRegex);
|
||||
|
||||
// 3. Wait snapshot finished
|
||||
waitForSinkSize("mongodb_sink", 2);
|
||||
|
||||
// 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
|
||||
// coll_b1.B102]
|
||||
insertRecordsInDatabase(db0);
|
||||
insertRecordsInDatabase(db1);
|
||||
|
||||
// 5. Wait change stream records come
|
||||
waitForSinkSize("mongodb_sink", 4);
|
||||
|
||||
// 6. Check results
|
||||
String[] expected =
|
||||
new String[] {
|
||||
String.format("+I[%s, coll_b1, B101]", db0),
|
||||
String.format("+I[%s, coll_b2, B201]", db0),
|
||||
String.format("+I[%s, coll_b1, B102]", db0),
|
||||
String.format("+I[%s, coll_b2, B202]", db0)
|
||||
};
|
||||
|
||||
List<String> actual = TestValuesTableFactory.getResults("mongodb_sink");
|
||||
assertThat(actual, containsInAnyOrder(expected));
|
||||
|
||||
result.getJobClient().get().cancel().get();
|
||||
}
|
||||
|
||||
/** match single database and multiple collections: database = db0 collection = .*coll_b\d? . */
|
||||
@Test
|
||||
public void testMatchSingleDatabaseWithCollectionPattern() throws Exception {
|
||||
// 1. Given collections:
|
||||
// db0: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db0 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
// db1: [coll_a1, coll_a2, coll_b1, coll_b2]
|
||||
String db1 = executeCommandFileInSeparateDatabase("ns_regex");
|
||||
|
||||
// 2. Test match: collection .*coll_b\d?
|
||||
String collectionRegex = ".*coll_b\\d?";
|
||||
TableResult result = submitTestCase(db0, collectionRegex);
|
||||
|
||||
// 3. Wait snapshot finished
|
||||
waitForSinkSize("mongodb_sink", 2);
|
||||
|
||||
// 4. Insert new records in database: [coll_a1.A102, coll_a2.A202, coll_b1.B102,
|
||||
// coll_b1.B102]
|
||||
insertRecordsInDatabase(db0);
|
||||
insertRecordsInDatabase(db1);
|
||||
|
||||
// 5. Wait change stream records come
|
||||
waitForSinkSize("mongodb_sink", 4);
|
||||
|
||||
// 6. Check results
|
||||
String[] expected =
|
||||
new String[] {
|
||||
String.format("+I[%s, coll_b1, B101]", db0),
|
||||
String.format("+I[%s, coll_b2, B201]", db0),
|
||||
String.format("+I[%s, coll_b1, B102]", db0),
|
||||
String.format("+I[%s, coll_b2, B202]", db0)
|
||||
};
|
||||
|
||||
List<String> actual = TestValuesTableFactory.getResults("mongodb_sink");
|
||||
assertThat(actual, containsInAnyOrder(expected));
|
||||
|
||||
result.getJobClient().get().cancel().get();
|
||||
}
|
||||
|
||||
private TableResult submitTestCase(String database, String collection) throws Exception {
|
||||
String sourceDDL =
|
||||
"CREATE TABLE mongodb_source ("
|
||||
+ " _id STRING NOT NULL,"
|
||||
+ " seq STRING,"
|
||||
+ " db_name STRING METADATA FROM 'database_name' VIRTUAL,"
|
||||
+ " coll_name STRING METADATA FROM 'collection_name' VIRTUAL,"
|
||||
+ " PRIMARY KEY (_id) NOT ENFORCED"
|
||||
+ ") WITH ("
|
||||
+ ignoreIfNull("hosts", MONGODB_CONTAINER.getHostAndPort())
|
||||
+ ignoreIfNull("username", FLINK_USER)
|
||||
+ ignoreIfNull("password", FLINK_USER_PASSWORD)
|
||||
+ ignoreIfNull("database", database)
|
||||
+ ignoreIfNull("collection", collection)
|
||||
+ " 'connector' = 'mongodb-cdc'"
|
||||
+ ")";
|
||||
|
||||
String sinkDDL =
|
||||
"CREATE TABLE mongodb_sink ("
|
||||
+ " db_name STRING,"
|
||||
+ " coll_name STRING,"
|
||||
+ " seq STRING,"
|
||||
+ " PRIMARY KEY (db_name, coll_name, seq) NOT ENFORCED"
|
||||
+ ") WITH ("
|
||||
+ " 'connector' = 'values',"
|
||||
+ " 'sink-insert-only' = 'false'"
|
||||
+ ")";
|
||||
tEnv.executeSql(sourceDDL);
|
||||
tEnv.executeSql(sinkDDL);
|
||||
|
||||
// async submit job
|
||||
TableResult result =
|
||||
tEnv.executeSql(
|
||||
"INSERT INTO mongodb_sink SELECT db_name, coll_name, seq FROM mongodb_source");
|
||||
|
||||
waitForSnapshotStarted("mongodb_sink");
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private String ignoreIfNull(String configName, String configValue) {
|
||||
return configValue != null ? String.format(" '%s' = '%s',", configName, configValue) : "";
|
||||
}
|
||||
|
||||
private void insertRecordsInDatabase(String database) {
|
||||
MongoDatabase db = getMongoDatabase(database);
|
||||
db.getCollection("coll_a1").insertOne(new Document("seq", "A102"));
|
||||
db.getCollection("coll_a2").insertOne(new Document("seq", "A202"));
|
||||
db.getCollection("coll_b1").insertOne(new Document("seq", "B102"));
|
||||
db.getCollection("coll_b2").insertOne(new Document("seq", "B202"));
|
||||
}
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.ververica.cdc.connectors.mongodb.utils;
|
||||
|
||||
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/** MongoDB test utilities. */
|
||||
public class MongoDBTestUtils {
|
||||
|
||||
public static void waitForSnapshotStarted(String sinkName) throws InterruptedException {
|
||||
while (sinkSize(sinkName) == 0) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
}
|
||||
|
||||
public static void waitForSinkSize(String sinkName, int expectedSize)
|
||||
throws InterruptedException {
|
||||
waitForSinkSize(sinkName, expectedSize, 10, TimeUnit.MINUTES);
|
||||
}
|
||||
|
||||
public static void waitForSinkSize(
|
||||
String sinkName, int expectedSize, long timeout, TimeUnit timeUnit)
|
||||
throws InterruptedException {
|
||||
long deadline = System.nanoTime() + timeUnit.toNanos(timeout);
|
||||
while (sinkSize(sinkName) < expectedSize) {
|
||||
if (System.nanoTime() > deadline) {
|
||||
fail(
|
||||
"Wait for sink size timeout, raw results: \n"
|
||||
+ String.join(
|
||||
"\n", TestValuesTableFactory.getRawResults(sinkName)));
|
||||
}
|
||||
Thread.sleep(100);
|
||||
}
|
||||
}
|
||||
|
||||
public static int sinkSize(String sinkName) {
|
||||
synchronized (TestValuesTableFactory.class) {
|
||||
try {
|
||||
return TestValuesTableFactory.getRawResults(sinkName).size();
|
||||
} catch (IllegalArgumentException e) {
|
||||
// job is not started yet
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,19 @@
|
||||
// Licensed to the Apache Software Foundation (ASF) under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
// Unless required by applicable law or agreed to in writing,
|
||||
// software distributed under the License is distributed on an
|
||||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
// KIND, either express or implied. See the License for the
|
||||
// specific language governing permissions and limitations
|
||||
// under the License.
|
||||
|
||||
db.getCollection('coll_a1').insertOne({"seq": "A101"});
|
||||
db.getCollection('coll_a2').insertOne({"seq": "A201"});
|
||||
db.getCollection('coll_b1').insertOne({"seq": "B101"});
|
||||
db.getCollection('coll_b2').insertOne({"seq": "B201"});
|
@ -0,0 +1,31 @@
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
# Unless required by applicable law or agreed to in writing,
|
||||
# software distributed under the License is distributed on an
|
||||
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
# KIND, either express or implied. See the License for the
|
||||
# specific language governing permissions and limitations
|
||||
# under the License.
|
||||
|
||||
XK8G9pNKhEPp/BlsKT7pHEc5i0oCpvNVZMALH5pD/6EHSuMzuyO1FpoeDwmWHXl0
|
||||
+Gp+VOI89Xp7E6eqop+fFHtoM3Mnk2oTiI/442GvS0xISPTwFVY9nO3MfO8VcPVx
|
||||
J3JCAb80GeXD5x55eAOi7NqXzpjk0OKqfPEwIn1lrjlkL2m5vq6kaKEd93i1+bMh
|
||||
3LRd1jLbgwWWxqYVV92BTQNnJin+G1er7Y2FzLpeFIKqyy+I22qIE2XIC7yj3wSw
|
||||
kxwKsPN5LjFsfVeKpf169R0KgBg4Nm0qlllVUGNKuEjaVoLOEBOJgoPnhC6L2avc
|
||||
/iDeunZDlDDgYG6t6aJXJelP+W1uXp4JQj1j18Scn0lrvgWxdAVrAtK6ftxqutHc
|
||||
RQBt6Ap63zojTraulm3aeo/w/yz0zjyYjxQ5t8cojIM/7TaNLe2GfVxwhqitUPL1
|
||||
ct2YFXWwX1H/+8E7yTsnquKqe6+r0aGQqxS5x+wFMsDun/1mxv7jgjwzZc1rEk8H
|
||||
DGdhnQ7MFPOE6Bp03zGpa6B6K4I5uDgUUeOC7zmAN63cPEumuuCjPVK42sMt5wwR
|
||||
NPJyL4+sWHa9vb2sBJ1dk3thQ+wwz856BZ9ILgeMUutQgasSwctlI7t3rhM+BGYy
|
||||
+naEhKWN9/cIDXtl3ZMhNWJIh/MqbluYazQ/97MZHeWc9CJXFU6yUrnJOdE0VvQd
|
||||
tROQNDuEB0Tq9ITxSYpZTY49+1CQp5E14GIc8frieWPvcbNVknriFquQfsW/tMvk
|
||||
V2Aj8sBYE+sW9sGQJlyfRrhTSN6aBG1em7ZkOAgcx2/5ftaEZTwBxNnJR9VZDYEi
|
||||
CDbobs3hIX3qhS6J9YbTEPFF2L6MMTL3ADgS44cWtmlYQrb2HJT0YLmdCzk4lSa6
|
||||
yWYLorduRtblgGo6v/nn7y41gn/l/aRdcDUsii/LgMco4ZPSRm0HixD8oA3agX9/
|
||||
23M5UVNCBO4/RKFOnjWM/2tN1xjeQrS2Hn6j3BtoTOl6k4ho
|
Loading…
Reference in New Issue