[mongodb] Support metadata columns for mongodb-cdc connector (#476)
parent
51f5a90eb3
commit
65665ebb6a
@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.internal;
|
||||
|
||||
/**
|
||||
* An immutable descriptor for the structure of {@link
|
||||
* com.mongodb.client.model.changestream.ChangeStreamDocument} envelopes.
|
||||
*/
|
||||
public class MongoDBEnvelope {
|
||||
|
||||
public static final String CLUSTER_TIME_FIELD = "clusterTime";
|
||||
|
||||
public static final String FULL_DOCUMENT_FIELD = "fullDocument";
|
||||
|
||||
public static final String DOCUMENT_KEY_FIELD = "documentKey";
|
||||
|
||||
public static final String OPERATION_TYPE_FIELD = "operationType";
|
||||
|
||||
public static final String NAMESPACE_FIELD = "ns";
|
||||
|
||||
public static final String NAMESPACE_DATABASE_FIELD = "db";
|
||||
|
||||
public static final String NAMESPACE_COLLECTION_FIELD = "coll";
|
||||
}
|
@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.table.api.DataTypes;
|
||||
import org.apache.flink.table.data.StringData;
|
||||
import org.apache.flink.table.data.TimestampData;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
|
||||
import com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope;
|
||||
import com.ververica.cdc.debezium.table.MetadataConverter;
|
||||
import io.debezium.connector.AbstractSourceInfo;
|
||||
import io.debezium.data.Envelope;
|
||||
import org.apache.kafka.connect.data.Struct;
|
||||
import org.apache.kafka.connect.source.SourceRecord;
|
||||
|
||||
/** Defines the supported metadata columns for {@link MongoDBTableSource}. */
|
||||
public enum MongoDBReadableMetadata {
|
||||
|
||||
/** Name of the collection that contain the row. */
|
||||
COLLECTION(
|
||||
"collection_name",
|
||||
DataTypes.STRING().notNull(),
|
||||
new MetadataConverter() {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@Override
|
||||
public Object read(SourceRecord record) {
|
||||
Struct value = (Struct) record.value();
|
||||
Struct to = value.getStruct(MongoDBEnvelope.NAMESPACE_FIELD);
|
||||
return StringData.fromString(
|
||||
to.getString(MongoDBEnvelope.NAMESPACE_COLLECTION_FIELD));
|
||||
}
|
||||
}),
|
||||
|
||||
/** Name of the database that contain the row. */
|
||||
DATABASE(
|
||||
"database_name",
|
||||
DataTypes.STRING().notNull(),
|
||||
new MetadataConverter() {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@Override
|
||||
public Object read(SourceRecord record) {
|
||||
Struct value = (Struct) record.value();
|
||||
Struct to = value.getStruct(MongoDBEnvelope.NAMESPACE_FIELD);
|
||||
return StringData.fromString(
|
||||
to.getString(MongoDBEnvelope.NAMESPACE_DATABASE_FIELD));
|
||||
}
|
||||
}),
|
||||
|
||||
/**
|
||||
* It indicates the time that the change was made in the database. If the record is read from
|
||||
* snapshot of the table instead of the change stream, the value is always 0.
|
||||
*/
|
||||
OP_TS(
|
||||
"op_ts",
|
||||
DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(),
|
||||
new MetadataConverter() {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@Override
|
||||
public Object read(SourceRecord record) {
|
||||
Struct value = (Struct) record.value();
|
||||
Struct source = value.getStruct(Envelope.FieldName.SOURCE);
|
||||
return TimestampData.fromEpochMillis(
|
||||
(Long) source.get(AbstractSourceInfo.TIMESTAMP_KEY));
|
||||
}
|
||||
});
|
||||
|
||||
private final String key;
|
||||
|
||||
private final DataType dataType;
|
||||
|
||||
private final MetadataConverter converter;
|
||||
|
||||
MongoDBReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
|
||||
this.key = key;
|
||||
this.dataType = dataType;
|
||||
this.converter = converter;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public DataType getDataType() {
|
||||
return dataType;
|
||||
}
|
||||
|
||||
public MetadataConverter getConverter() {
|
||||
return converter;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue