DBZ-2948 Implement BLOB/CLOB support for Oracle LogMiner

This commit is contained in:
Chris Cranford 2021-05-07 03:30:55 -04:00 committed by Gunnar Morling
parent ba581ed323
commit 49703d9dc9
30 changed files with 7896 additions and 283 deletions

View File

@ -0,0 +1,27 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.util.ArrayList;
import java.util.List;
/**
* A "marker" class for passing a collection of Blob data type chunks to {@link OracleValueConverters}
* so that each chunk can be converted, decoded, and combined into a single binary representation
* for event emission.
*
* @author Chris Cranford
*/
public class BlobChunkList extends ArrayList<String> {
/**
* Creates a BLOB chunk list backed by the provided collection.
*
* @param backingList collection of BLOB chunks
*/
public BlobChunkList(List<String> backingList) {
super(backingList);
}
}

View File

@ -5,7 +5,6 @@
*/
package io.debezium.connector.oracle;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
@ -340,23 +339,31 @@ public Scn getCurrentScn() throws SQLException {
});
}
public OracleConnection executeLegacy(String... sqlStatements) throws SQLException {
return executeLegacy(statement -> {
for (String sqlStatement : sqlStatements) {
if (sqlStatement != null) {
statement.execute(sqlStatement);
}
}
});
}
public OracleConnection executeLegacy(Operations operations) throws SQLException {
Connection conn = connection();
try (Statement statement = conn.createStatement()) {
operations.apply(statement);
commit();
/**
* Get a byte-array value from a given {@code HEXTORAW} argument.
*
* The provided {@code hexToRawValue} may be provided as the direct hex-string argument or the entire
* value may be wrapped with the {@code HEXTORAW} function call, which will be omitted.
*
* @param hexToRawValue the hex-to-raw string, never {@code null}
* @return byte array of decoded value, may be {@code null}
* @throws SQLException if there is a database exception
*/
public byte[] getHexToRawByteArray(String hexToRawValue) throws SQLException {
final String data;
if (hexToRawValue.startsWith("HEXTORAW('") && hexToRawValue.endsWith("')")) {
data = hexToRawValue.substring(10, hexToRawValue.length() - 2);
}
return this;
else {
data = hexToRawValue;
}
return prepareQueryAndMap("select HEXTORAW(?) FROM DUAL", ps -> ps.setString(1, data), rs -> {
if (rs.next()) {
return rs.getBytes(1);
}
return null;
});
}
public static String connectionString(Configuration config) {

View File

@ -8,6 +8,8 @@
import static io.debezium.util.NumberConversions.BYTE_FALSE;
import java.math.BigDecimal;
import java.nio.Buffer;
import java.nio.ByteBuffer;
import java.sql.Blob;
import java.sql.Clob;
import java.sql.SQLException;
@ -20,6 +22,8 @@
import java.time.format.DateTimeFormatter;
import java.time.format.DateTimeFormatterBuilder;
import java.time.temporal.ChronoField;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -254,7 +258,10 @@ protected Object convertString(Column column, Field fieldDefn, Object data) {
}
if (data instanceof String) {
String s = (String) data;
if (s.startsWith("UNISTR('") && s.endsWith("')")) {
if (s.equals("EMPTY_CLOB()")) {
return null;
}
else if (s.startsWith("UNISTR('") && s.endsWith("')")) {
return convertOracleUnistr(column, fieldDefn, s.substring(8, s.length() - 2));
}
}
@ -290,16 +297,49 @@ private String convertOracleUnistr(Column column, Field fieldDefn, String data)
@Override
protected Object convertBinary(Column column, Field fieldDefn, Object data, BinaryHandlingMode mode) {
if (data instanceof Blob) {
try {
try {
if (data instanceof String) {
String str = (String) data;
if (str.equals("EMPTY_BLOB()")) {
return null;
}
else if (str.startsWith("HEXTORAW('") && str.endsWith("')")) {
data = connection.getHexToRawByteArray(str);
}
}
else if (data instanceof BlobChunkList) {
data = convertBlobChunkList((BlobChunkList) data);
}
else if (data instanceof Blob) {
Blob blob = (Blob) data;
return blob.getBytes(0, Long.valueOf(blob.length()).intValue());
}
catch (SQLException e) {
throw new DebeziumException("Couldn't convert value for column " + column.name(), e);
data = blob.getBytes(1, Long.valueOf(blob.length()).intValue());
}
return super.convertBinary(column, fieldDefn, data, mode);
}
return super.convertBinary(column, fieldDefn, data, mode);
catch (SQLException e) {
throw new DebeziumException("Couldn't convert value for column " + column.name(), e);
}
}
private byte[] convertBlobChunkList(BlobChunkList chunks) throws SQLException {
if (chunks.isEmpty()) {
// if there are no chunks, simply return null
return null;
}
// Iterate each chunk's hex-string and generate a ByteBuffer for each chunk
// Oracle internal string limits enforce doing the chunk processing one-at-a-time.
List<ByteBuffer> buffers = new ArrayList<>(chunks.size());
for (String chunk : chunks) {
buffers.add(ByteBuffer.wrap(connection.getHexToRawByteArray(chunk)));
}
// Combine ByteBuffer instances into a single ByteBuffer
ByteBuffer buffer = ByteBuffer.allocate(buffers.stream().mapToInt(Buffer::remaining).sum());
buffers.forEach(b -> buffer.put(b.duplicate()));
return buffer.array();
}
@Override

View File

@ -71,7 +71,6 @@ public StreamingChangeEventSource getSource(OffsetContext offsetContext,
errorHandler,
clock,
schema,
taskContext,
jdbcConfig,
streamingMetrics);
}

View File

@ -8,6 +8,7 @@
import java.util.Arrays;
import java.util.List;
import io.debezium.DebeziumException;
import io.debezium.connector.oracle.BaseChangeRecordEmitter;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
@ -21,7 +22,7 @@
*/
public class LogMinerChangeRecordEmitter extends BaseChangeRecordEmitter<LogMinerColumnValue> {
private LogMinerDmlEntry dmlEntry;
private final LogMinerDmlEntry dmlEntry;
protected final Table table;
public LogMinerChangeRecordEmitter(OffsetContext offset, LogMinerDmlEntry dmlEntry, Table table, Clock clock) {
@ -32,7 +33,17 @@ public LogMinerChangeRecordEmitter(OffsetContext offset, LogMinerDmlEntry dmlEnt
@Override
protected Operation getOperation() {
return dmlEntry.getCommandType();
switch (dmlEntry.getOperation()) {
case RowMapper.INSERT:
return Operation.CREATE;
case RowMapper.UPDATE:
case RowMapper.SELECT_LOB_LOCATOR:
return Operation.UPDATE;
case RowMapper.DELETE:
return Operation.DELETE;
default:
throw new DebeziumException("Unsupported operation type: " + dmlEntry.getOperation());
}
}
@Override

View File

@ -27,14 +27,13 @@
import io.debezium.connector.oracle.logminer.parser.DmlParser;
import io.debezium.connector.oracle.logminer.parser.DmlParserException;
import io.debezium.connector.oracle.logminer.parser.LogMinerDmlParser;
import io.debezium.connector.oracle.logminer.parser.SelectLobParser;
import io.debezium.connector.oracle.logminer.parser.SimpleDmlParser;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.data.Envelope.Operation;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.util.Clock;
/**
* This class process entries obtained from LogMiner view.
@ -56,8 +55,8 @@ class LogMinerQueryResultProcessor {
private final OracleDatabaseSchema schema;
private final EventDispatcher<TableId> dispatcher;
private final OracleConnectorConfig connectorConfig;
private final Clock clock;
private final HistoryRecorder historyRecorder;
private final SelectLobParser selectLobParser;
private Scn currentOffsetScn = Scn.NULL;
private Scn currentOffsetCommitScn = Scn.NULL;
@ -68,17 +67,17 @@ class LogMinerQueryResultProcessor {
TransactionalBuffer transactionalBuffer,
OracleOffsetContext offsetContext, OracleDatabaseSchema schema,
EventDispatcher<TableId> dispatcher,
Clock clock, HistoryRecorder historyRecorder) {
HistoryRecorder historyRecorder) {
this.context = context;
this.streamingMetrics = streamingMetrics;
this.transactionalBuffer = transactionalBuffer;
this.offsetContext = offsetContext;
this.schema = schema;
this.dispatcher = dispatcher;
this.clock = clock;
this.historyRecorder = historyRecorder;
this.connectorConfig = connectorConfig;
this.dmlParser = resolveParser(connectorConfig, schema.getValueConverters());
this.selectLobParser = new SelectLobParser();
}
private static DmlParser resolveParser(OracleConnectorConfig connectorConfig, OracleValueConverters valueConverters) {
@ -91,7 +90,6 @@ private static DmlParser resolveParser(OracleConnectorConfig connectorConfig, Or
/**
* This method does all the job
* @param resultSet the info from LogMiner view
* @return number of processed DMLs from the given resultSet
* @throws SQLException thrown if any database exception occurs
*/
void processResult(ResultSet resultSet) throws SQLException {
@ -104,6 +102,10 @@ void processResult(ResultSet resultSet) throws SQLException {
rows++;
Scn scn = RowMapper.getScn(resultSet);
if (scn.isNull()) {
throw new DebeziumException("Unexpected null SCN detected in LogMiner results");
}
String tableName = RowMapper.getTableName(resultSet);
String segOwner = RowMapper.getSegOwner(resultSet);
int operationCode = RowMapper.getOperationCode(resultSet);
@ -113,12 +115,12 @@ void processResult(ResultSet resultSet) throws SQLException {
String userName = RowMapper.getUsername(resultSet);
String rowId = RowMapper.getRowId(resultSet);
int rollbackFlag = RowMapper.getRollbackFlag(resultSet);
int sequence = RowMapper.getSequence(resultSet);
Object rsId = RowMapper.getRsId(resultSet);
long hash = RowMapper.getHash(resultSet);
boolean dml = isDmlOperation(operationCode);
boolean isDml = false;
if (operationCode == RowMapper.INSERT || operationCode == RowMapper.UPDATE || operationCode == RowMapper.DELETE) {
isDml = true;
}
String redoSql = RowMapper.getSqlRedo(resultSet, isDml, historyRecorder, scn, tableName, segOwner, operationCode, changeTime, txId);
String redoSql = RowMapper.getSqlRedo(resultSet, dml, historyRecorder, scn, tableName, segOwner, operationCode, changeTime, txId);
LOGGER.trace("scn={}, operationCode={}, operation={}, table={}, segOwner={}, userName={}, rowId={}, rollbackFlag={}", scn, operationCode, operation,
tableName, segOwner, userName, rowId, rollbackFlag);
@ -126,114 +128,139 @@ void processResult(ResultSet resultSet) throws SQLException {
String logMessage = String.format("transactionId=%s, SCN=%s, table_name=%s, segOwner=%s, operationCode=%s, offsetSCN=%s, " +
" commitOffsetSCN=%s", txId, scn, tableName, segOwner, operationCode, offsetContext.getScn(), offsetContext.getCommitScn());
if (scn.isNull()) {
LogMinerHelper.logWarn(streamingMetrics, "Scn is null for {}", logMessage);
return;
}
// Commit
if (operationCode == RowMapper.COMMIT) {
if (transactionalBuffer.isTransactionRegistered(txId)) {
switch (operationCode) {
case RowMapper.START: {
// Register start transaction.
// If already registered, does nothing due to overlapping mining strategy.
transactionalBuffer.registerTransaction(txId, scn);
break;
}
case RowMapper.COMMIT: {
// Commits a transaction
if (transactionalBuffer.isTransactionRegistered(txId)) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
if (transactionalBuffer.commit(txId, scn, offsetContext, changeTime, context, logMessage, dispatcher)) {
LOGGER.trace("COMMIT, {}", logMessage);
commitCounter++;
}
}
break;
}
case RowMapper.ROLLBACK: {
// Rollback a transaction
if (transactionalBuffer.isTransactionRegistered(txId)) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
if (transactionalBuffer.rollback(txId, logMessage)) {
LOGGER.trace("ROLLBACK, {}", logMessage);
rollbackCounter++;
}
}
break;
}
case RowMapper.DDL: {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
}
if (transactionalBuffer.commit(txId, scn, offsetContext, changeTime, context, logMessage, dispatcher)) {
LOGGER.trace("COMMIT, {}", logMessage);
commitCounter++;
}
continue;
}
// Rollback
if (operationCode == RowMapper.ROLLBACK) {
if (transactionalBuffer.isTransactionRegistered(txId)) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
}
if (transactionalBuffer.rollback(txId, logMessage)) {
LOGGER.trace("ROLLBACK, {}", logMessage);
rollbackCounter++;
}
continue;
}
// DDL
if (operationCode == RowMapper.DDL) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
LOGGER.info("DDL: {}, REDO_SQL: {}", logMessage, redoSql);
try {
if (tableName != null) {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
dispatcher.dispatchSchemaChangeEvent(tableId,
new OracleSchemaChangeEventEmitter(
connectorConfig,
offsetContext,
tableId,
tableId.catalog(),
tableId.schema(),
redoSql,
schema,
changeTime.toInstant(),
streamingMetrics));
LOGGER.info("DDL: {}, REDO_SQL: {}", logMessage, redoSql);
try {
if (tableName != null) {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
dispatcher.dispatchSchemaChangeEvent(tableId,
new OracleSchemaChangeEventEmitter(
connectorConfig,
offsetContext,
tableId,
tableId.catalog(),
tableId.schema(),
redoSql,
schema,
changeTime.toInstant(),
streamingMetrics));
}
}
catch (InterruptedException e) {
throw new DebeziumException("Failed to dispatch DDL event", e);
}
}
catch (InterruptedException e) {
throw new DebeziumException("Failed to dispatch DDL event", e);
case RowMapper.MISSING_SCN: {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
LogMinerHelper.logWarn(streamingMetrics, "Missing SCN, {}", logMessage);
break;
}
continue;
}
// MISSING_SCN
if (operationCode == RowMapper.MISSING_SCN) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, 0, redoSql);
LogMinerHelper.logWarn(streamingMetrics, "Missing SCN, {}", logMessage);
continue;
}
// DML
if (isDml) {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
LOGGER.trace("DML, {}, sql {}", logMessage, redoSql);
if (redoSql == null) {
LOGGER.trace("Redo SQL was empty, DML operation skipped.");
continue;
case RowMapper.SELECT_LOB_LOCATOR: {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
final LogMinerDmlEntry entry = selectLobParser.parse(redoSql);
entry.setObjectOwner(segOwner);
entry.setSourceTime(changeTime);
entry.setTransactionId(txId);
entry.setObjectName(tableName);
entry.setScn(scn);
entry.setRowId(rowId);
entry.setSequence(sequence);
transactionalBuffer.registerSelectLobOperation(operationCode, txId, scn, tableId, entry,
selectLobParser.getColumnName(), selectLobParser.isBinary(), changeTime.toInstant(), rowId, rsId, hash);
break;
}
dmlCounter++;
switch (operationCode) {
case RowMapper.INSERT:
insertCounter++;
break;
case RowMapper.UPDATE:
updateCounter++;
break;
case RowMapper.DELETE:
deleteCounter++;
break;
case RowMapper.LOB_WRITE: {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
transactionalBuffer.registerLobWriteOperation(operationCode, txId, scn, tableId, redoSql,
changeTime.toInstant(), rowId, rsId, hash);
break;
}
final Table table = schema.tableFor(tableId);
if (table == null) {
LogMinerHelper.logWarn(streamingMetrics, "DML for table '{}' that is not known to this connector, skipping.", tableId);
continue;
case RowMapper.LOB_ERASE: {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
transactionalBuffer.registerLobEraseOperation(operationCode, txId, scn, tableId, changeTime.toInstant(), rowId, rsId, hash);
break;
}
case RowMapper.INSERT:
case RowMapper.UPDATE:
case RowMapper.DELETE: {
LOGGER.trace("DML, {}, sql {}", logMessage, redoSql);
if (redoSql != null) {
final TableId tableId = RowMapper.getTableId(connectorConfig.getCatalogName(), resultSet);
dmlCounter++;
switch (operationCode) {
case RowMapper.INSERT:
insertCounter++;
break;
case RowMapper.UPDATE:
updateCounter++;
break;
case RowMapper.DELETE:
deleteCounter++;
break;
}
if (rollbackFlag == 1) {
// DML operation is to undo partial or all operations as a result of a rollback.
// This can be situations where an insert or update causes a constraint violation
// and a subsequent operation is written to the logs to revert the change.
transactionalBuffer.undoDmlOperation(txId, rowId, tableId);
continue;
final Table table = schema.tableFor(tableId);
if (table == null) {
LogMinerHelper.logWarn(streamingMetrics, "DML for table '{}' that is not known to this connector, skipping.", tableId);
continue;
}
if (rollbackFlag == 1) {
// DML operation is to undo partial or all operations as a result of a rollback.
// This can be situations where an insert or update causes a constraint violation
// and a subsequent operation is written to the logs to revert the change.
transactionalBuffer.undoDmlOperation(txId, rowId, tableId);
continue;
}
final LogMinerDmlEntry dmlEntry = parse(redoSql, table, txId);
dmlEntry.setObjectOwner(segOwner);
dmlEntry.setSourceTime(changeTime);
dmlEntry.setTransactionId(txId);
dmlEntry.setObjectName(tableName);
dmlEntry.setScn(scn);
dmlEntry.setRowId(rowId);
dmlEntry.setSequence(sequence);
transactionalBuffer.registerDmlOperation(operationCode, txId, scn, tableId, dmlEntry,
changeTime.toInstant(), rowId, rsId, hash);
}
else {
LOGGER.trace("Redo SQL was empty, DML operation skipped.");
}
break;
}
final LogMinerDmlEntry dmlEntry = parse(redoSql, table, txId);
dmlEntry.setObjectOwner(segOwner);
dmlEntry.setSourceTime(changeTime);
dmlEntry.setTransactionId(txId);
dmlEntry.setObjectName(tableName);
dmlEntry.setScn(scn);
dmlEntry.setRowId(rowId);
transactionalBuffer.registerDmlOperation(operationCode, txId, scn, tableId, dmlEntry, changeTime.toInstant(), rowId);
}
}
@ -311,7 +338,7 @@ private LogMinerDmlEntry parse(String redoSql, Table table, String txId) {
}
if (dmlEntry.getOldValues().isEmpty()) {
if (Operation.UPDATE.equals(dmlEntry.getCommandType()) || Operation.DELETE.equals(dmlEntry.getCommandType())) {
if (RowMapper.UPDATE == dmlEntry.getOperation() || RowMapper.DELETE == dmlEntry.getOperation()) {
LOGGER.warn("The DML event '{}' contained no before state.", redoSql);
streamingMetrics.incrementWarningCount();
}
@ -319,4 +346,15 @@ private LogMinerDmlEntry parse(String redoSql, Table table, String txId) {
return dmlEntry;
}
private static boolean isDmlOperation(int operationCode) {
switch (operationCode) {
case RowMapper.INSERT:
case RowMapper.UPDATE:
case RowMapper.DELETE:
return true;
default:
return false;
}
}
}

View File

@ -46,7 +46,6 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.Scn;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.pipeline.ErrorHandler;
@ -74,7 +73,6 @@ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventS
private final Set<String> racHosts = new HashSet<>();
private final JdbcConfiguration jdbcConfiguration;
private final OracleConnectorConfig.LogMiningStrategy strategy;
private final OracleTaskContext taskContext;
private final ErrorHandler errorHandler;
private final boolean isContinuousMining;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
@ -88,8 +86,7 @@ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventS
public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig, OracleOffsetContext offsetContext,
OracleConnection jdbcConnection, EventDispatcher<TableId> dispatcher,
ErrorHandler errorHandler, Clock clock, OracleDatabaseSchema schema,
OracleTaskContext taskContext, Configuration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
Configuration jdbcConfig, OracleStreamingChangeEventSourceMetrics streamingMetrics) {
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
this.clock = clock;
@ -99,7 +96,6 @@ public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
this.strategy = connectorConfig.getLogMiningStrategy();
this.isContinuousMining = connectorConfig.isContinuousMining();
this.errorHandler = errorHandler;
this.taskContext = taskContext;
this.streamingMetrics = streamingMetrics;
this.jdbcConfiguration = JdbcConfiguration.adapt(jdbcConfig);
this.isRac = connectorConfig.isRacSystem();
@ -131,7 +127,7 @@ public void execute(ChangeEventSourceContext context) {
setNlsSessionParameters(jdbcConnection);
checkSupplementalLogging(jdbcConnection, connectorConfig.getPdbName(), schema);
initializeRedoLogsForMining(jdbcConnection, false, archiveLogRetention);
initializeRedoLogsForMining(jdbcConnection, false, archiveLogRetention, startScn);
HistoryRecorder historyRecorder = connectorConfig.getLogMiningHistoryRecorder();
@ -141,7 +137,7 @@ public void execute(ChangeEventSourceContext context) {
final LogMinerQueryResultProcessor processor = new LogMinerQueryResultProcessor(context, jdbcConnection,
connectorConfig, streamingMetrics, transactionalBuffer, offsetContext, schema, dispatcher,
clock, historyRecorder);
historyRecorder);
final String query = SqlUtils.logMinerContentsQuery(connectorConfig, jdbcConnection.username());
try (PreparedStatement miningView = jdbcConnection.connection().prepareStatement(query, ResultSet.TYPE_FORWARD_ONLY,
@ -165,7 +161,7 @@ public void execute(ChangeEventSourceContext context) {
startScn, endScn, offsetContext.getScn(), strategy, isContinuousMining);
endMining(jdbcConnection);
initializeRedoLogsForMining(jdbcConnection, true, archiveLogRetention);
initializeRedoLogsForMining(jdbcConnection, true, archiveLogRetention, startScn);
abandonOldTransactionsIfExist(jdbcConnection, transactionalBuffer);
@ -174,8 +170,13 @@ public void execute(ChangeEventSourceContext context) {
currentRedoLogSequences = getCurrentRedoLogSequences();
}
startLogMining(jdbcConnection, startScn, endScn, strategy, isContinuousMining, streamingMetrics);
Scn miningStartScn = offsetContext.getCommitScn();
if (miningStartScn == null) {
miningStartScn = offsetContext.getScn();
}
startLogMining(jdbcConnection, miningStartScn, endScn, strategy, isContinuousMining, streamingMetrics);
LOGGER.trace("Fetching LogMiner view results SCN {} to {}", startScn, endScn);
stopwatch.start();
miningView.setFetchSize(connectorConfig.getMaxQueueSize());
miningView.setFetchDirection(ResultSet.FETCH_FORWARD);
@ -185,13 +186,7 @@ public void execute(ChangeEventSourceContext context) {
Duration lastDurationOfBatchCapturing = stopwatch.stop().durations().statistics().getTotal();
streamingMetrics.setLastDurationOfBatchCapturing(lastDurationOfBatchCapturing);
processor.processResult(rs);
startScn = endScn;
if (transactionalBuffer.isEmpty()) {
LOGGER.debug("Transactional buffer empty, updating offset's SCN {}", startScn);
offsetContext.setScn(startScn);
}
startScn = transactionalBuffer.updateOffsetContext(offsetContext);
}
streamingMetrics.setCurrentBatchProcessingTime(Duration.between(start, Instant.now()));
@ -228,7 +223,7 @@ private void abandonOldTransactionsIfExist(OracleConnection connection, Transact
}
}
private void initializeRedoLogsForMining(OracleConnection connection, boolean postEndMiningSession, Duration archiveLogRetention) throws SQLException {
private void initializeRedoLogsForMining(OracleConnection connection, boolean postEndMiningSession, Duration archiveLogRetention, Scn startScn) throws SQLException {
if (!postEndMiningSession) {
if (OracleConnectorConfig.LogMiningStrategy.CATALOG_IN_REDO.equals(strategy)) {
buildDataDictionary(connection);

View File

@ -33,10 +33,15 @@ public class RowMapper {
public static final int DELETE = 2;
public static final int UPDATE = 3;
public static final int DDL = 5;
public static final int START = 6;
public static final int COMMIT = 7;
public static final int SELECT_LOB_LOCATOR = 9;
public static final int LOB_WRITE = 10;
public static final int LOB_ERASE = 29;
public static final int MISSING_SCN = 34;
public static final int ROLLBACK = 36;
// columns mapped by index based on query in SqlUtils#logMinerContentsQuery
private static final int SCN = 1;
private static final int SQL_REDO = 2;
private static final int OPERATION_CODE = 3;
@ -49,12 +54,15 @@ public class RowMapper {
private static final int USERNAME = 10;
private static final int ROW_ID = 11;
private static final int ROLLBACK_FLAG = 12;
private static final int SEQUENCE = 13;
private static final int RS_ID = 14;
private static final int HASH = 15;
private static final Calendar UTC_CALENDAR = Calendar.getInstance(TimeZone.getTimeZone(ZoneOffset.UTC));
// todo: add these for recording
// private static final int SESSION_NUMBER = 10;
// private static final int SERIAL_NUMBER = 11;
// private static final int RS_ID = 12;
// private static final int SSN = 12;
public static String getOperation(ResultSet rs) throws SQLException {
return rs.getString(OPERATION);
@ -111,6 +119,7 @@ public static String getTransactionId(ResultSet rs) throws SQLException {
* @param changeTime time of change
* @param txId transaction ID
* @return the redo SQL
* @throws SQLException if an exception occurred while interacting with the data source
*/
public static String getSqlRedo(ResultSet rs, boolean isDml, HistoryRecorder historyRecorder, Scn scn, String tableName,
String segOwner, int operationCode, Timestamp changeTime, String txId)
@ -137,10 +146,12 @@ public static String getSqlRedo(ResultSet rs, boolean isDml, HistoryRecorder his
LOGGER.warn("LOB value was truncated due to the connector limitation of {} MB", 40);
break;
}
result.append(rs.getString(SQL_REDO));
redoSql = rs.getString(SQL_REDO);
result.append(redoSql);
csf = rs.getInt(CSF);
if (isDml) {
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, csf, rs.getString(SQL_REDO));
historyRecorder.record(scn, tableName, segOwner, operationCode, changeTime, txId, csf, redoSql);
}
}
@ -159,4 +170,15 @@ public static TableId getTableId(String catalogName, ResultSet rs) throws SQLExc
return new TableId(catalogName, rs.getString(SEG_OWNER), rs.getString(TABLE_NAME));
}
public static int getSequence(ResultSet rs) throws SQLException {
return rs.getInt(SEQUENCE);
}
public static Object getRsId(ResultSet rs) throws SQLException {
return rs.getObject(RS_ID);
}
public static long getHash(ResultSet rs) throws SQLException {
return rs.getLong(HASH);
}
}

View File

@ -8,7 +8,6 @@
import java.io.IOException;
import java.sql.SQLRecoverableException;
import java.time.Duration;
import java.time.LocalDateTime;
import java.util.Iterator;
import java.util.List;
import java.util.regex.Pattern;
@ -282,7 +281,8 @@ static String startLogMinerStatement(Scn startScn, Scn endScn, OracleConnectorCo
*/
static String logMinerContentsQuery(OracleConnectorConfig connectorConfig, String logMinerUser) {
StringBuilder query = new StringBuilder();
query.append("SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ROW_ID, ROLLBACK ");
query.append("SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ");
query.append("ROW_ID, ROLLBACK, SEQUENCE#, RS_ID, ORA_HASH(SCN||OPERATION||RS_ID||SEQUENCE#||RTRIM(SUBSTR(SQL_REDO,1,256))) ");
query.append("FROM ").append(LOGMNR_CONTENTS_VIEW).append(" ");
query.append("WHERE ");
query.append("SCN > ? AND SCN <= ? ");
@ -290,9 +290,9 @@ static String logMinerContentsQuery(OracleConnectorConfig connectorConfig, Strin
// MISSING_SCN/DDL only when not performed by excluded users
// For DDL, the `INTERNAL DDL%` info rows should be excluded as these are commands executed by the database that
// typically perform operations such as renaming a deleted object when dropped if the drop doesn't specify PURGE
query.append("(OPERATION_CODE IN (5,34) AND USERNAME NOT IN (").append(getExcludedUsers(logMinerUser)).append(") AND INFO NOT LIKE 'INTERNAL DDL%') ");
query.append("(OPERATION_CODE IN (5,9,10,11,29,34) AND USERNAME NOT IN (").append(getExcludedUsers(logMinerUser)).append(") AND INFO NOT LIKE 'INTERNAL DDL%') ");
// COMMIT/ROLLBACK
query.append("OR (OPERATION_CODE IN (7,36)) ");
query.append("OR (OPERATION_CODE IN (6,7,36)) ");
// INSERT/UPDATE/DELETE
query.append("OR ");
query.append("(OPERATION_CODE IN (1,2,3) ");
@ -468,18 +468,4 @@ public static boolean connectionProblem(Throwable e) {
e.getMessage().toUpperCase().contains("CONNECTION IS CLOSED") ||
e.getMessage().toUpperCase().startsWith("NO MORE DATA TO READ FROM SOCKET");
}
public static long parseRetentionFromName(String historyTableName) {
String[] tokens = historyTableName.split("_");
if (tokens.length != 7) {
return 0;
}
LocalDateTime recorded = LocalDateTime.of(
Integer.parseInt(tokens[2]), // year
Integer.parseInt(tokens[3]), // month
Integer.parseInt(tokens[4]), // days
Integer.parseInt(tokens[5]), // hours
Integer.parseInt(tokens[6])); // minutes
return Duration.between(recorded, LocalDateTime.now()).toHours();
}
}

View File

@ -16,20 +16,26 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Supplier;
import org.apache.kafka.connect.errors.DataException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.DebeziumException;
import io.debezium.annotation.NotThreadSafe;
import io.debezium.connector.oracle.BlobChunkList;
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValueImpl;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.ChangeEventSource;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
import io.debezium.util.Clock;
@ -50,13 +56,17 @@ public final class TransactionalBuffer implements AutoCloseable {
private final ErrorHandler errorHandler;
private final Set<String> abandonedTransactionIds;
private final Set<String> rolledBackTransactionIds;
private final Set<RecentlyCommittedTransaction> recentlyCommittedTransactionIds;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private Scn lastCommittedScn;
private Scn maxCommittedScn;
/**
* Constructor to create a new instance.
*
* @param schema database schema
* @param clock system clock
* @param errorHandler the connector error handler
* @param streamingMetrics the streaming metrics
*/
@ -66,8 +76,10 @@ public final class TransactionalBuffer implements AutoCloseable {
this.clock = clock;
this.errorHandler = errorHandler;
this.lastCommittedScn = Scn.NULL;
this.maxCommittedScn = Scn.NULL;
this.abandonedTransactionIds = new HashSet<>();
this.rolledBackTransactionIds = new HashSet<>();
this.recentlyCommittedTransactionIds = new HashSet<>();
this.streamingMetrics = streamingMetrics;
}
@ -88,25 +100,82 @@ Set<String> getRolledBackTransactionIds() {
* @param parseEntry parser entry
* @param changeTime time the DML operation occurred
* @param rowId unique row identifier
* @param rsId rollback sequence identifier
* @param hash unique row hash
*/
void registerDmlOperation(int operation, String transactionId, Scn scn, TableId tableId, LogMinerDmlEntry parseEntry, Instant changeTime, String rowId) {
if (abandonedTransactionIds.contains(transactionId)) {
LogMinerHelper.logWarn(streamingMetrics, "Captured DML for abandoned transaction {}, ignored.", transactionId);
return;
}
if (rolledBackTransactionIds.contains(transactionId)) {
LogMinerHelper.logWarn(streamingMetrics, "Captured DML for rolled back transaction {}, ignored.", transactionId);
return;
}
Transaction transaction = transactions.computeIfAbsent(transactionId, s -> new Transaction(transactionId, scn));
transaction.events.add(new DmlEvent(operation, parseEntry, scn, tableId, rowId));
void registerDmlOperation(int operation, String transactionId, Scn scn, TableId tableId, LogMinerDmlEntry parseEntry,
Instant changeTime, String rowId, Object rsId, long hash) {
registerEvent(transactionId, scn, hash, () -> new DmlEvent(operation, parseEntry, scn, tableId, rowId, rsId));
streamingMetrics.setActiveTransactions(transactions.size());
streamingMetrics.incrementRegisteredDmlCount();
streamingMetrics.calculateLagMetrics(changeTime);
}
/**
* Register a {@code SEL_LOB_LOCATOR} operation with the transaction buffer.
*
* @param operation operation type
* @param transactionId unique transaction identifier
* @param scn system change number
* @param tableId table identifier
* @param parseEntry parser entry
* @param changeTime time the operation occurred
* @param rowId unique row identifier
* @param rsId rollback sequence identifier
* @param hash unique row hash
*/
void registerSelectLobOperation(int operation, String transactionId, Scn scn, TableId tableId, LogMinerDmlEntry parseEntry,
String columnName, boolean binaryData, Instant changeTime, String rowId, Object rsId, long hash) {
registerEvent(transactionId, scn, hash, () -> new SelectLobLocatorEvent(operation, parseEntry, columnName, binaryData, scn, tableId, rowId, rsId));
streamingMetrics.setActiveTransactions(transactions.size());
streamingMetrics.calculateLagMetrics(changeTime);
}
/**
* Register a {@code LOB_WRITE} operation with the transaction buffer.
*
* @param operation operation type
* @param transactionId unique transaction identifier
* @param scn system change number
* @param tableId table identifier
* @param data data written by the LOB operation
* @param changeTime time the operation occurred
* @param rowId unique row identifier
* @param rsId rollback sequence identifier
* @param hash unique row hash
*/
void registerLobWriteOperation(int operation, String transactionId, Scn scn, TableId tableId, String data,
Instant changeTime, String rowId, Object rsId, long hash) {
if (data != null) {
final String sql = parseLobWriteSql(data);
registerEvent(transactionId, scn, hash, () -> new LobWriteEvent(operation, sql, scn, tableId, rowId, rsId));
streamingMetrics.setActiveTransactions(transactions.size());
streamingMetrics.calculateLagMetrics(changeTime);
}
}
/**
* Register a {@code LOB_ERASE} operation with the transction buffer.
*
* @param operation operation type
* @param transactionId unique transaction identifier
* @param scn system change number
* @param tableId table identifier
* @param changeTime time the operation occurred
* @param rowId unique row identifier
* @param rsId rollback sequence identifier
* @param hash unique row hash
*/
void registerLobEraseOperation(int operation, String transactionId, Scn scn, TableId tableId, Instant changeTime,
String rowId, Object rsId, long hash) {
registerEvent(transactionId, scn, hash, () -> new LobEraseEvent(operation, scn, tableId, rowId, rsId));
streamingMetrics.setActiveTransactions(transactions.size());
streamingMetrics.calculateLagMetrics(changeTime);
}
/**
* Undo a staged DML operation in the transaction buffer.
*
@ -130,6 +199,20 @@ void undoDmlOperation(String transactionId, String undoRowId, TableId tableId) {
});
}
/**
* Register a new transaction with the transaction buffer.
*
* @param transactionId unique transaction identifier
* @param scn starting SCN of the transaction
*/
void registerTransaction(String transactionId, Scn scn) {
Transaction transaction = transactions.get(transactionId);
if (transaction == null) {
transactions.put(transactionId, new Transaction(transactionId, scn));
streamingMetrics.setActiveTransactions(transactions.size());
}
}
/**
* Commits a transaction by looking up the transaction in the buffer and if exists, all registered callbacks
* will be executed in chronological order, emitting events for each followed by a transaction commit event.
@ -166,19 +249,14 @@ boolean commit(String transactionId, Scn scn, OracleOffsetContext offsetContext,
return false;
}
reconcileTransaction(transaction);
LOGGER.trace("COMMIT, {}, smallest SCN: {}", debugMessage, smallestScn);
commit(context, offsetContext, start, transaction, timestamp, smallestScn, scn, dispatcher);
return true;
}
private void commit(ChangeEventSource.ChangeEventSourceContext context, OracleOffsetContext offsetContext, Instant start,
Transaction transaction, Timestamp timestamp, Scn smallestScn, Scn scn, EventDispatcher<TableId> dispatcher) {
try {
int counter = transaction.events.size();
for (DmlEvent event : transaction.events) {
for (LogMinerEvent event : transaction.events) {
if (!context.isRunning()) {
return;
return false;
}
// Update SCN in offset context only if processed SCN less than SCN among other transactions
@ -186,6 +264,7 @@ private void commit(ChangeEventSource.ChangeEventSourceContext context, OracleOf
offsetContext.setScn(event.getScn());
streamingMetrics.setOldestScn(event.getScn());
}
offsetContext.setTransactionId(transaction.transactionId);
offsetContext.setSourceTime(timestamp.toInstant());
offsetContext.setTableId(event.getTableId());
@ -193,23 +272,31 @@ private void commit(ChangeEventSource.ChangeEventSourceContext context, OracleOf
offsetContext.setCommitScn(scn);
}
LOGGER.trace("Processing DML event {} with SCN {}", event.getEntry(), event.getScn());
LOGGER.trace("Processing event {}", event);
dispatcher.dispatchDataChangeEvent(event.getTableId(),
new LogMinerChangeRecordEmitter(
offsetContext,
event.getEntry(),
schema.tableFor(event.getTableId()),
clock));
}
lastCommittedScn = Scn.valueOf(scn.longValue());
if (!transaction.events.isEmpty()) {
dispatcher.dispatchTransactionCommittedEvent(offsetContext);
}
if (lastCommittedScn.compareTo(maxCommittedScn) > 0) {
LOGGER.trace("Updated transaction buffer max commit SCN to '{}'", lastCommittedScn);
maxCommittedScn = lastCommittedScn;
}
// cache recent transaction and commit scn for handling offset updates
recentlyCommittedTransactionIds.add(new RecentlyCommittedTransaction(transaction, scn));
}
catch (InterruptedException e) {
LogMinerHelper.logError(streamingMetrics, "Thread interrupted during running", e);
LogMinerHelper.logError(streamingMetrics, "Commit interrupted", e);
Thread.currentThread().interrupt();
}
catch (Exception e) {
@ -223,6 +310,33 @@ private void commit(ChangeEventSource.ChangeEventSourceContext context, OracleOf
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.setLastCommitDuration(Duration.between(start, Instant.now()));
}
return true;
}
/**
* Update the offset context based on the current state of the transaction buffer.
*
* @param offsetContext offset context, should not be {@code null}
* @return offset context SCN, never {@code null}
*/
Scn updateOffsetContext(OracleOffsetContext offsetContext) {
if (transactions.isEmpty()) {
LOGGER.trace("Transaction buffer is empty, updating offset SCN to '{}'", maxCommittedScn);
offsetContext.setScn(maxCommittedScn);
}
else {
Scn minStartScn = transactions.values().stream().map(t -> t.firstScn).min(Scn::compareTo).orElse(Scn.NULL);
if (!minStartScn.isNull()) {
LOGGER.trace("Removing all commits up to SCN '{}'", minStartScn);
recentlyCommittedTransactionIds.removeIf(t -> t.firstScn.compareTo(minStartScn) < 0);
offsetContext.setScn(minStartScn.subtract(Scn.valueOf(1)));
}
else {
LOGGER.trace("Minimum SCN in transaction buffer is still SCN '{}'", minStartScn);
}
}
return offsetContext.getScn();
}
/**
@ -322,10 +436,521 @@ public String toString() {
@Override
public void close() {
transactions.clear();
}
if (this.streamingMetrics != null) {
// if metrics registered, unregister them
this.streamingMetrics.unregister(LOGGER);
/**
* Helper method to register a given {@link LogMinerEvent} implementation with the buffer.
*
* @param transactionId transaction id that contained the given event
* @param scn system change number for the event
* @param supplier supplier function to generate the event if validity checks pass
*/
private void registerEvent(String transactionId, Scn scn, long hash, Supplier<LogMinerEvent> supplier) {
if (abandonedTransactionIds.contains(transactionId)) {
LogMinerHelper.logWarn(streamingMetrics, "Event for abandoned transaction {}, ignored.", transactionId);
return;
}
if (rolledBackTransactionIds.contains(transactionId)) {
LogMinerHelper.logWarn(streamingMetrics, "Event for rolled back transaction {}, ignored.", transactionId);
return;
}
if (recentlyCommittedTransactionIds.contains(transactionId)) {
LOGGER.trace("Event for transaction {} skipped, transaction already committed.", transactionId);
return;
}
Transaction transaction = transactions.computeIfAbsent(transactionId, s -> new Transaction(transactionId, scn));
// Event will only be registered with transaction if the computed hash doesn't already exist.
// This is necessary to handle overlapping mining session scopes
if (!transaction.eventHashes.contains(hash)) {
transaction.eventHashes.add(hash);
transaction.events.add(supplier.get());
}
}
/**
* Parses a {@code LOB_WRITE} operation SQL fragment.
*
* @param sql sql statement
* @return the parsed statement
* @throws DebeziumException if an unexpected SQL fragment is provided that cannot be parsed
*/
private String parseLobWriteSql(String sql) {
if (sql == null) {
return null;
}
int start = sql.indexOf(":= '");
if (start != -1) {
// LOB_WRITE SQL is for a CLOB field
int end = sql.lastIndexOf("'");
return sql.substring(start + 4, end);
}
start = sql.indexOf(":= HEXTORAW");
if (start != -1) {
// LOB_WRITE SQL is for a BLOB field
int end = sql.lastIndexOf("'") + 2;
return sql.substring(start + 3, end);
}
throw new DebeziumException("Unable to parse unsupported LOB_WRITE SQL: " + sql);
}
/**
* Reconcile the specified transaction by merging multiple events that should be emitted as a single
* logical event, such as changes made to LOB column types that involve multiple events.
*
* @param transaction transaction to be reconciled, never {@code null}
*/
private void reconcileTransaction(Transaction transaction) {
LOGGER.trace("Reconciling transaction {}", transaction.transactionId);
LogMinerEvent prevEvent = null;
int prevEventSize = transaction.events.size();
for (int i = 0; i < transaction.events.size();) {
final LogMinerEvent event = transaction.events.get(i);
LOGGER.trace("Processing event {}", event);
switch (event.getOperation()) {
case RowMapper.SELECT_LOB_LOCATOR:
if (shouldMergeSelectLobLocatorEvent(transaction, i, (SelectLobLocatorEvent) event, prevEvent)) {
continue;
}
break;
case RowMapper.INSERT:
case RowMapper.UPDATE:
if (shouldMergeDmlEvent(transaction, i, (DmlEvent) event, prevEvent)) {
continue;
}
break;
}
++i;
prevEvent = event;
LOGGER.trace("Previous event is now {}", prevEvent);
}
if (transaction.events.size() != prevEventSize) {
LOGGER.trace("Reconciled transaction {} from {} events to {}.", transaction.transactionId, prevEventSize, transaction.events.size());
}
else {
LOGGER.trace("Transaction {} event queue was unmodified.", transaction.transactionId);
}
}
/**
* Attempts to merge the provided SEL_LOB_LOCATOR event with the previous event in the transaction.
*
* @param transaction transaction being processed, never {@code null}
* @param index event index being processed
* @param event event being processed, never {@code null}
* @param prevEvent previous event in the transaction, can be {@code null}
* @return true if the event is merged, false if the event was not merged.
*/
private boolean shouldMergeSelectLobLocatorEvent(Transaction transaction, int index, SelectLobLocatorEvent event, LogMinerEvent prevEvent) {
LOGGER.trace("\tDetected SelectLobLocatorEvent for column '{}'", event.getColumnName());
// Read and combine all LOB_WRITE events that follow SEL_LOB_LOCATOR
Object lobData = null;
final List<String> lobWrites = readAndCombineLobWriteEvents(transaction, index, event.isBinaryData());
if (!lobWrites.isEmpty()) {
if (event.isBinaryData()) {
// For BLOB we pass the list of string chunks as-is to the value converter
lobData = new BlobChunkList(lobWrites);
}
else {
// For CLOB we go ahead and pre-process the List into a single string.
lobData = String.join("", lobWrites);
}
}
// Read and consume all LOB_ERASE events that follow SEL_LOB_LOCATOR
final int lobEraseEvents = readAndConsumeLobEraseEvents(transaction, index);
if (lobEraseEvents > 0) {
LOGGER.warn("LOB_ERASE for table '{}' column '{}' is not supported, use DML operations to manipulate LOB columns only.", event.getTableId(),
event.getColumnName());
if (lobWrites.isEmpty()) {
// There are no write and only erase events, discard entire SEL_LOB_LOCATOR
// To simulate this, we treat this as a "merge" op so caller doesn't modify previous event
transaction.events.remove(index);
return true;
}
}
else if (lobEraseEvents == 0 && lobWrites.isEmpty()) {
// There were no LOB operations present, discard entire SEL_LOB_LOCATOR
// To simulate this, we treat this as a "merge" op so caller doesn't modify previous event
transaction.events.remove(index);
return true;
}
// SelectLobLocatorEvent can be treated as a parent DML operation where an update occurs on any
// LOB-based column. In this case, the event will be treated as an UPDATE event when emitted.
if (prevEvent == null) {
// There is no prior event, add column to this SelectLobLocatorEvent and don't merge.
LOGGER.trace("\tAdding column '{}' to current event", event.getColumnName());
event.getEntry().getNewValues().add(createColumnValue(event.getColumnName(), lobData));
return false;
}
if (RowMapper.INSERT == prevEvent.getOperation()) {
// Previous event is an INSERT operation.
// Only merge the SEL_LOB_LOCATOR event if the previous INSERT is for the same table/row
// and if the INSERT's column value is EMPTY_CLOB() or EMPTY_BLOB()
if (isForSameTableOrScn(event, prevEvent)) {
LOGGER.trace("\tMerging SEL_LOB_LOCATOR with previous INSERT event");
LogMinerColumnValue prevColumn = getColumnByName(prevEvent.getEntry().getNewValues(), event.getColumnName());
if (prevColumn == null || (!prevColumn.isEmptyClobFunction() && !prevColumn.isEmptyBlobFunction())) {
throw new DebeziumException("Expected to find column '" + event.getColumnName() + "' in table '"
+ prevEvent.getTableId() + "' to be initialized as an empty LOB value.'");
}
prevColumn.setColumnData(lobData);
// Remove the SEL_LOB_LOCATOR event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
else if (RowMapper.UPDATE == prevEvent.getOperation()) {
// Previous event is an UPDATE operation.
// Only merge the SEL_LOB_LOCATOR event if the previous UPDATE is for the same table/row
if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) {
LOGGER.trace("\tMerging SEL_LOB_LOCATOR with previous UPDATE event");
LogMinerColumnValue prevColumn = getColumnByName(prevEvent.getEntry().getNewValues(), event.getColumnName());
if (prevColumn == null) {
LOGGER.trace("\tAdding SEL_LOB_LOCATOR column '{}' to previous UPDATE event", event.getColumnName());
prevEvent.getEntry().getNewValues().add(createColumnValue(event.getColumnName(), lobData));
}
else {
LOGGER.trace("\tMerging SEL_LOB_LOCATOR column '{}' to previous UPDATE event", event.getColumnName());
prevColumn.setColumnData(lobData);
}
// Remove the SEL_LOB_LOCATOR event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
else if (RowMapper.SELECT_LOB_LOCATOR == prevEvent.getOperation()) {
// Previous event is a SEL_LOB_LOCATOR operation.
// Only merge the two SEL_LOB_LOCATOR events if they're for the same table/row
if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) {
LOGGER.trace("\tAdding column '{}' to previous SEL_LOB_LOCATOR event", event.getColumnName());
prevEvent.getEntry().getNewValues().add(createColumnValue(event.getColumnName(), lobData));
// Remove the SEL_LOB_LOCATOR event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
else {
throw new DebeziumException("Unexpected previous event operation: " + prevEvent.getOperation());
}
LOGGER.trace("\tSEL_LOB_LOCATOR event is for different row, merge skipped.");
LOGGER.trace("\tAdding column '{}' to current event", event.getColumnName());
event.getEntry().getNewValues().add(createColumnValue(event.getColumnName(), lobData));
return false;
}
/**
* Attempts to merge the provided DML event with the previous event in the transaction.
*
* @param transaction transaction being processed, never {@code null}
* @param index event index being processed
* @param event event being processed, never {@code null}
* @param prevEvent previous event in the transaction, can be {@ode null}
* @return true if the event is merged, false if the event was not merged
*/
private boolean shouldMergeDmlEvent(Transaction transaction, int index, DmlEvent event, LogMinerEvent prevEvent) {
LOGGER.trace("\tDetected DmlEvent {}", event.getOperation());
if (prevEvent == null) {
// There is no prior event, therefore there is no reason to perform any merge.
return false;
}
if (RowMapper.INSERT == prevEvent.getOperation()) {
// Previous event is an INSERT operation.
// The only valid combination here would be if the current event is an UPDATE since an INSERT cannot
// be merged with a prior INSERT with how LogMiner materializes the rows.
if (RowMapper.UPDATE == event.getOperation()) {
if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) {
LOGGER.trace("\tMerging UPDATE event with previous INSERT event");
mergeNewColumns(event, prevEvent);
// Remove the UPDATE event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
}
else if (RowMapper.UPDATE == prevEvent.getOperation()) {
// Previous event is an UPDATE operation.
// This will happen if there are non-CLOB and inline-CLOB fields updated in the same SQL.
// The inline-CLOB values should be merged with the previous UPDATE event.
if (RowMapper.UPDATE == event.getOperation()) {
if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) {
LOGGER.trace("\tMerging UPDATE event with previous UPDATE event");
mergeNewColumns(event, prevEvent);
// Remove the UPDATE event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
}
else if (RowMapper.SELECT_LOB_LOCATOR == prevEvent.getOperation()) {
// Previous event is a SEL_LOB_LOCATOR operation.
// SQL contained both non-inline CLOB and inline-CLOB field changes.
if (RowMapper.UPDATE == event.getOperation()) {
if (isForSameTableOrScn(event, prevEvent) && isSameTableRow(event, prevEvent)) {
LOGGER.trace("\tMerging UPDATE event with previous SEL_LOB_LOCATOR event");
for (LogMinerColumnValue value : event.getEntry().getNewValues()) {
boolean found = false;
for (LogMinerColumnValue prevValue : prevEvent.getEntry().getNewValues()) {
if (prevValue.getColumnName().equals(value.getColumnName())) {
found = true;
break;
}
}
if (!found && value.getColumnData() != null) {
LOGGER.trace("\tAdding column '{}' to previous SEL_LOB_LOCATOR event", value.getColumnName());
prevEvent.getEntry().getNewValues().add(value);
}
}
// Remove the UPDATE event from event list and indicate merged.
transaction.events.remove(index);
return true;
}
}
}
LOGGER.trace("\tDmlEvent {} event is for different row, merge skipped.", event.getOperation());
return false;
}
/**
* Reads the transaction event queue and combines all LOB_WRITE events starting at the provided index.
* for a SEL_LOB_LOCATOR event which is for binary data (BLOB) data types.
*
* @param transaction transaction being processed, never {@code null}
* @param index index to the first LOB_WRITE operation
* @return list of string-based values for each LOB_WRITE operation
*/
private List<String> readAndCombineLobWriteEvents(Transaction transaction, int index, boolean binaryData) {
List<String> chunks = new ArrayList<>();
for (int i = index + 1; i < transaction.events.size(); ++i) {
final LogMinerEvent event = transaction.events.get(i);
if (!(event instanceof LobWriteEvent)) {
break;
}
final LobWriteEvent writeEvent = (LobWriteEvent) event;
if (binaryData && !writeEvent.getData().startsWith("HEXTORAW('") && !writeEvent.getData().endsWith("')")) {
throw new DebeziumException("Unexpected BLOB data chunk: " + writeEvent.getData());
}
chunks.add(writeEvent.getData());
}
if (!chunks.isEmpty()) {
LOGGER.trace("\tCombined {} LobWriteEvent events", chunks.size());
// Remove events from the transaction queue queue
for (int i = 0; i < chunks.size(); ++i) {
transaction.events.remove(index + 1);
}
}
return chunks;
}
/**
* Read and remove all LobErase events detected in the transaction event queue.
*
* @param transaction transaction being processed, never {@code null}
* @param index index to the first LOB_ERASE operation
* @return number of LOB_ERASE events consumed and removed from the event queue
*/
private int readAndConsumeLobEraseEvents(Transaction transaction, int index) {
int events = 0;
for (int i = index + 1; i < transaction.events.size(); ++i) {
final LogMinerEvent event = transaction.events.get(i);
if (!(event instanceof LobEraseEvent)) {
break;
}
events++;
}
if (events > 0) {
LOGGER.trace("\tConsumed {} LobErase events", events);
for (int i = 0; i < events; ++i) {
transaction.events.remove(index + 1);
}
}
return events;
}
/**
* Utility method to create a {@link LogMinerColumnValue} from a provided column name nad value.
*
* @param columnName column name, never {@code null}
* @param columnValue column value, may be {@code null}
* @return the constructed {@link LogMinerColumnValue} instance
*/
private LogMinerColumnValue createColumnValue(String columnName, Object columnValue) {
final LogMinerColumnValueImpl column = new LogMinerColumnValueImpl(columnName);
column.setColumnData(columnValue);
return column;
}
/**
* Checks whether the two events are for the same table or participate in the same system change.
*
* @param event current event being processed, never {@code null}
* @param prevEvent previous/parent event that has been processed, may be {@code null}
* @return true if the two events are for the same table or system change number, false otherwise
*/
private boolean isForSameTableOrScn(LogMinerEvent event, LogMinerEvent prevEvent) {
if (prevEvent != null) {
if (event.getTableId().equals(prevEvent.getTableId())) {
return true;
}
return event.getScn().equals(prevEvent.getScn()) && event.getRsId().equals(prevEvent.getRsId());
}
return false;
}
/**
* Checks whether the two events are for the same table row.
*
* @param event current event being processed, never {@code null}
* @param prevEvent previous/parent event that has been processed, never {@code null}
* @return true if the two events are for the same table row, false otherwise
*/
private boolean isSameTableRow(LogMinerEvent event, LogMinerEvent prevEvent) {
final Table table = schema.tableFor(event.getTableId());
if (table == null) {
LOGGER.trace("Unable to locate table '{}' schema, unable to detect if same row.", event.getTableId());
return false;
}
for (String columnName : table.primaryKeyColumnNames()) {
LogMinerColumnValue prevValue = getColumnByName(prevEvent.getEntry().getNewValues(), columnName);
if (prevValue == null) {
throw new DebeziumException("Could not find column " + columnName + " in previous event");
}
LogMinerColumnValue value = getColumnByName(event.getEntry().getNewValues(), columnName);
if (value == null) {
throw new DebeziumException("Could not find column " + columnName + " in event");
}
if (!Objects.equals(value.getColumnData(), prevValue.getColumnData())) {
return false;
}
}
return true;
}
/**
* Merge {@link LogMinerColumnValue} instances from provided {@code event} with {@code prevEvent}.
*
* @param event current event being processed, never {@code null}
* @param prevEvent previous/parent parent that has been processed, never {@code null}
*/
private void mergeNewColumns(LogMinerEvent event, LogMinerEvent prevEvent) {
final boolean prevEventIsInsert = RowMapper.INSERT == prevEvent.getOperation();
for (LogMinerColumnValue value : event.getEntry().getNewValues()) {
boolean found = false;
for (LogMinerColumnValue prevValue : prevEvent.getEntry().getNewValues()) {
if (prevValue.getColumnName().equals(value.getColumnName())) {
if (prevEventIsInsert && prevValue.isEmptyClobFunction()) {
LOGGER.trace("\tAssigning column '{}' with updated CLOB value.", prevValue.getColumnName());
prevValue.setColumnData(value.getColumnData());
}
else if (prevEventIsInsert && prevValue.isEmptyBlobFunction()) {
LOGGER.trace("\tAssigning column '{}' with updated BLOB value.", prevValue.getColumnName());
prevValue.setColumnData(value.getColumnData());
}
else if (!prevEventIsInsert && value.getColumnData() != null) {
LOGGER.trace("\tUpdating column '{}' in previous event", prevValue.getColumnName());
prevValue.setColumnData(value.getColumnData());
}
found = true;
break;
}
}
if (!found) {
if (prevEventIsInsert) {
// INSERTs as previous events should have a previous value of EMPTY_CLOB()
// If they don't we should throw an exception for sanity sake
throw new DebeziumException("Expected to find column '" + value.getColumnName() + "' with EMPTY_CLOB()");
}
// UPDATEs as previous events where we don't find the column will simply be added
prevEvent.getEntry().getNewValues().add(value);
}
}
}
/**
* Locate a {@link LogMinerColumnValue} in a collection by column name.
*
* @param columns collection of LogMiner column values
* @param columnName name of the column to find
* @return column value instance for the specified column name, may be {@code null} if no column with name exists
*/
private static LogMinerColumnValue getColumnByName(List<LogMinerColumnValue> columns, String columnName) {
return columns.stream().filter(c -> c.getColumnName().equals(columnName)).findFirst().orElse(null);
}
/**
* Represents a transaction boundary that was recently committed.
*
* This is used by the buffer to detect transactions read from overlapping mining sessions that can
* safely be ignored as the connector has already reconciled and emitted the event for it.
*/
private static final class RecentlyCommittedTransaction {
private final String transactionId;
private final Scn firstScn;
private final Scn commitScn;
public RecentlyCommittedTransaction(Transaction transaction, Scn commitScn) {
this.transactionId = transaction.transactionId;
this.firstScn = transaction.firstScn;
this.commitScn = commitScn;
}
public Scn getFirstScn() {
return firstScn;
}
public Scn getCommitScn() {
return commitScn;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
RecentlyCommittedTransaction that = (RecentlyCommittedTransaction) o;
return Objects.equals(transactionId, that.transactionId) &&
Objects.equals(firstScn, that.firstScn) &&
Objects.equals(commitScn, that.commitScn);
}
@Override
public int hashCode() {
return Objects.hash(transactionId, firstScn, commitScn);
}
}
@ -337,12 +962,14 @@ private static final class Transaction {
private final String transactionId;
private final Scn firstScn;
private Scn lastScn;
private final List<DmlEvent> events;
private final Set<Long> eventHashes;
private final List<LogMinerEvent> events;
private Transaction(String transactionId, Scn firstScn) {
this.transactionId = transactionId;
this.firstScn = firstScn;
this.events = new ArrayList<>();
this.eventHashes = new HashSet<>();
this.lastScn = firstScn;
}
@ -357,20 +984,22 @@ public String toString() {
}
/**
* Represents a DML event for a given table row.
* Base class for all possible LogMiner events
*/
private static class DmlEvent {
private static class LogMinerEvent {
private final int operation;
private final LogMinerDmlEntry entry;
private final Scn scn;
private final TableId tableId;
private final String rowId;
private final Object rsId;
public DmlEvent(int operation, LogMinerDmlEntry entry, Scn scn, TableId tableId, String rowId) {
public LogMinerEvent(int operation, LogMinerDmlEntry entry, Scn scn, TableId tableId, String rowId, Object rsId) {
this.operation = operation;
this.scn = scn;
this.tableId = tableId;
this.rowId = rowId;
this.rsId = rsId;
this.entry = entry;
}
@ -394,6 +1023,10 @@ public String getRowId() {
return rowId;
}
public Object getRsId() {
return rsId;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -402,17 +1035,76 @@ public boolean equals(Object o) {
if (o == null || getClass() != o.getClass()) {
return false;
}
DmlEvent dmlEvent = (DmlEvent) o;
return operation == dmlEvent.operation &&
Objects.equals(entry, dmlEvent.entry) &&
Objects.equals(scn, dmlEvent.scn) &&
Objects.equals(tableId, dmlEvent.tableId) &&
Objects.equals(rowId, dmlEvent.rowId);
LogMinerEvent event = (LogMinerEvent) o;
return operation == event.operation &&
Objects.equals(entry, event.entry) &&
Objects.equals(scn, event.scn) &&
Objects.equals(tableId, event.tableId) &&
Objects.equals(rowId, event.rowId) &&
Objects.equals(rsId, event.rsId);
}
@Override
public int hashCode() {
return Objects.hash(operation, entry, scn, tableId, rowId);
return Objects.hash(operation, entry, scn, tableId, rowId, rsId);
}
}
/**
* Represents a DML event for a given table row.
*/
private static class DmlEvent extends LogMinerEvent {
public DmlEvent(int operation, LogMinerDmlEntry entry, Scn scn, TableId tableId, String rowId, Object rsId) {
super(operation, entry, scn, tableId, rowId, rsId);
}
}
/**
* Represents a SELECT_LOB_LOCATOR event
*/
private static class SelectLobLocatorEvent extends LogMinerEvent {
private final String columnName;
private final boolean binaryData;
public SelectLobLocatorEvent(int operation, LogMinerDmlEntry entry, String columnName, boolean binaryData, Scn scn,
TableId tableId, String rowId, Object rsId) {
super(operation, entry, scn, tableId, rowId, rsId);
this.columnName = columnName;
this.binaryData = binaryData;
}
public String getColumnName() {
return columnName;
}
public boolean isBinaryData() {
return binaryData;
}
}
/**
* Represents a LOB_WRITE event
*/
private static class LobWriteEvent extends LogMinerEvent {
private final String data;
public LobWriteEvent(int operation, String data, Scn scn, TableId tableId, String rowId, Object rsId) {
super(operation, null, scn, tableId, rowId, rsId);
this.data = data;
}
public String getData() {
return data;
}
}
/**
* Represents a LOB_ERASE event
*/
private static class LobEraseEvent extends LogMinerEvent {
public LobEraseEvent(int operation, Scn scn, TableId tableId, String rowId, Object rsId) {
super(operation, null, scn, tableId, rowId, rsId);
}
}
}

View File

@ -12,11 +12,11 @@
import java.util.Map;
import io.debezium.DebeziumException;
import io.debezium.connector.oracle.logminer.RowMapper;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValueImpl;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntryImpl;
import io.debezium.data.Envelope;
import io.debezium.relational.Table;
/**
@ -126,7 +126,7 @@ private LogMinerDmlEntry parseInsert(String sql, Table table) {
newValues.add(createColumnValue(columnName, columnValue));
}
return new LogMinerDmlEntryImpl(Envelope.Operation.CREATE, newValues, Collections.emptyList());
return new LogMinerDmlEntryImpl(RowMapper.INSERT, newValues, Collections.emptyList());
}
catch (Exception e) {
throw new DmlParserException("Failed to parse insert DML: '" + sql + "'", e);
@ -190,12 +190,12 @@ private LogMinerDmlEntry parseUpdate(String sql, Table table) {
for (int i = 0; i < table.columns().size(); ++i) {
String columnName = table.columns().get(i).name();
if (afterColumnMap.containsKey(columnName)) {
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName, 0);
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName);
value.setColumnData(afterColumnMap.get(columnName));
newValues.add(value);
}
else {
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName, 0);
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName);
value.setColumnData(beforeColumnMap.get(columnName));
newValues.add(value);
}
@ -205,7 +205,7 @@ private LogMinerDmlEntry parseUpdate(String sql, Table table) {
newValues = Collections.emptyList();
}
return new LogMinerDmlEntryImpl(Envelope.Operation.UPDATE, newValues, oldValues);
return new LogMinerDmlEntryImpl(RowMapper.UPDATE, newValues, oldValues);
}
catch (Exception e) {
throw new DmlParserException("Failed to parse update DML: '" + sql + "'", e);
@ -247,7 +247,7 @@ private LogMinerDmlEntry parseDelete(String sql, Table table) {
oldValues = Collections.emptyList();
}
return new LogMinerDmlEntryImpl(Envelope.Operation.DELETE, Collections.emptyList(), oldValues);
return new LogMinerDmlEntryImpl(RowMapper.DELETE, Collections.emptyList(), oldValues);
}
catch (Exception e) {
throw new DmlParserException("Failed to parse delete DML: '" + sql + "'", e);
@ -639,7 +639,7 @@ private static String removeSingleQuotes(String text) {
* @return the LogMiner column value object
*/
private static LogMinerColumnValue createColumnValue(String columnName, String columnValue) {
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName, 0);
LogMinerColumnValue value = new LogMinerColumnValueImpl(columnName);
if (columnValue != null && !columnValue.equals(NULL)) {
value.setColumnData(columnValue);
}

View File

@ -0,0 +1,193 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.logminer.parser;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;
import io.debezium.annotation.NotThreadSafe;
import io.debezium.connector.oracle.logminer.RowMapper;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValueImpl;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntryImpl;
/**
* Simple text-based parser implementation for Oracle LogMiner SEL_LOB_LOCATOR Redo SQL.
*
* @author Chris Cranford
*/
@NotThreadSafe
public class SelectLobParser {
private static final String BEGIN = "BEGIN";
private static final String SELECT = "select";
private static final String FROM = "from";
private static final String WHERE = "where";
private static final String AND = "and";
private static final String OR = "or";
private static final String FOR_UPDATE = "for update";
private static final String BLOB_LOCATOR = "loc_b";
private static final String BLOB_BUFFER = "buf_b";
private String columnName;
private String schemaName;
private String tableName;
private boolean binary;
private List<LogMinerColumnValue> columns;
/**
* Parse the supplied SEL_LOB_LOCATOR event redo SQL statement.
*
* @param sql SQL statement expression to be parsed
* @return instance of {@link LogMinerDmlEntry} for the parsed fragment.
*/
public LogMinerDmlEntry parse(String sql) {
// Reset internal state
reset();
if (sql != null) {
int start = sql.indexOf(BEGIN);
if (start != -1) {
start = sql.indexOf(" ", start + 1) + 1;
if (sql.indexOf(SELECT, start) == start) {
start = sql.indexOf(" ", start) + 1;
start = parseQuotedValue(sql, start, s -> columnName = s);
start = sql.indexOf(" ", start) + 1; // skip leading space
start = sql.indexOf(" ", start) + 1; // skip into
if (sql.indexOf(BLOB_LOCATOR, start) == start || sql.indexOf(BLOB_BUFFER, start) == start) {
binary = true;
}
start = sql.indexOf(" ", start) + 1; // skip loc_xxxx variable name
if (sql.indexOf(FROM, start) == start) {
start = sql.indexOf(" ", start) + 1;
start = parseQuotedValue(sql, start, s -> schemaName = s);
if (sql.indexOf('.', start) == start) {
start += 1; // dot
start = parseQuotedValue(sql, start, s -> tableName = s);
start += 1; // space
if (sql.indexOf(WHERE, start) == start) {
start += WHERE.length() + 1;
parseWhere(sql, start);
}
}
}
}
}
}
LogMinerDmlEntryImpl entry = new LogMinerDmlEntryImpl(RowMapper.SELECT_LOB_LOCATOR, new ArrayList<>(columns), columns);
entry.setObjectOwner(schemaName);
entry.setObjectName(tableName);
return entry;
}
/**
* @return the column name that the SEL_LOB_LOCATOR event is modifying
*/
public String getColumnName() {
return columnName;
}
/**
* @return true if the column being modified is a BLOB; otherwise false indicates a CLOB data type.
*/
public boolean isBinary() {
return binary;
}
private int parseQuotedValue(String sql, int index, Consumer<String> collector) {
boolean inDoubleQuotes = false;
int start = -1, last = -1;
for (int i = index; i < sql.length(); ++i) {
if (sql.charAt(i) == '"') {
if (!inDoubleQuotes) {
inDoubleQuotes = true;
start = i + 1;
}
else {
inDoubleQuotes = false;
last = i;
index = i + 1;
break;
}
}
}
if (start != -1 && last != -1) {
collector.accept(sql.substring(start, last));
}
return index;
}
private int parseColumnValue(String sql, int index, Consumer<String> collector) {
boolean inSingleQuotes = false;
int start = -1, last = -1;
for (int i = index; i < sql.length(); ++i) {
char c = sql.charAt(i);
char lookAhead = (index + 1 < sql.length()) ? sql.charAt(i + 1) : 0;
if (c == '\'') {
// skip over double single quote
if (inSingleQuotes && lookAhead == '\'') {
index += 1;
continue;
}
if (inSingleQuotes) {
inSingleQuotes = false;
last = i;
index = i + 1;
break;
}
inSingleQuotes = true;
start = i + 1;
}
}
if (start != -1 && last != -1) {
collector.accept(sql.substring(start, last));
}
return index;
}
private int parseWhere(String sql, int index) {
for (int i = index; i < sql.length(); ++i) {
// parse column name
StringBuilder columnName = new StringBuilder();
index = parseQuotedValue(sql, index, columnName::append);
index += 3; // space, equals, space
final LogMinerColumnValueImpl column = new LogMinerColumnValueImpl(columnName.toString());
index = parseColumnValue(sql, index, column::setColumnData);
index += 1; // space
columns.add(column);
if (sql.indexOf(AND, index) == index) {
// 'and' detected
index += AND.length() + 1;
}
else if (sql.indexOf(OR, index) == index) {
// 'or' detecteed
index += OR.length() + 1;
}
else if (sql.indexOf(FOR_UPDATE, index) == index) {
// 'for update' detected
// this signifies the end of the where clause
break;
}
}
return index;
}
private void reset() {
columnName = null;
schemaName = null;
tableName = null;
binary = false;
columns = new ArrayList<>();
}
}

View File

@ -17,12 +17,12 @@
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.antlr.listener.ParserUtils;
import io.debezium.connector.oracle.logminer.RowMapper;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValueImpl;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValueWrapper;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntryImpl;
import io.debezium.data.Envelope;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
import io.debezium.text.ParsingException;
@ -104,21 +104,21 @@ public LogMinerDmlEntry parse(String dmlContent, Table table, String txId) {
.filter(LogMinerColumnValueWrapper::isProcessed).map(LogMinerColumnValueWrapper::getColumnValue).collect(Collectors.toList());
List<LogMinerColumnValue> actualOldValues = oldColumnValues.values().stream()
.filter(LogMinerColumnValueWrapper::isProcessed).map(LogMinerColumnValueWrapper::getColumnValue).collect(Collectors.toList());
return new LogMinerDmlEntryImpl(Envelope.Operation.UPDATE, actualNewValues, actualOldValues);
return new LogMinerDmlEntryImpl(RowMapper.UPDATE, actualNewValues, actualOldValues);
}
else if (st instanceof Insert) {
parseInsert(table, (Insert) st);
List<LogMinerColumnValue> actualNewValues = newColumnValues.values()
.stream().map(LogMinerColumnValueWrapper::getColumnValue).collect(Collectors.toList());
return new LogMinerDmlEntryImpl(Envelope.Operation.CREATE, actualNewValues, Collections.emptyList());
return new LogMinerDmlEntryImpl(RowMapper.INSERT, actualNewValues, Collections.emptyList());
}
else if (st instanceof Delete) {
parseDelete(table, (Delete) st);
List<LogMinerColumnValue> actualOldValues = oldColumnValues.values()
.stream().map(LogMinerColumnValueWrapper::getColumnValue).collect(Collectors.toList());
return new LogMinerDmlEntryImpl(Envelope.Operation.DELETE, Collections.emptyList(), actualOldValues);
return new LogMinerDmlEntryImpl(RowMapper.DELETE, Collections.emptyList(), actualOldValues);
}
else {
@ -137,11 +137,10 @@ private void initColumns(Table table, String tableName) {
this.table = table;
for (int i = 0; i < table.columns().size(); i++) {
Column column = table.columns().get(i);
int type = column.jdbcType();
String key = column.name();
String name = ParserUtils.stripeQuotes(column.name().toUpperCase());
newColumnValues.put(key, new LogMinerColumnValueWrapper(new LogMinerColumnValueImpl(name, type)));
oldColumnValues.put(key, new LogMinerColumnValueWrapper(new LogMinerColumnValueImpl(name, type)));
newColumnValues.put(key, new LogMinerColumnValueWrapper(new LogMinerColumnValueImpl(name)));
oldColumnValues.put(key, new LogMinerColumnValueWrapper(new LogMinerColumnValueImpl(name)));
}
}

View File

@ -18,6 +18,16 @@ public interface LogMinerColumnValue {
*/
String getColumnName();
/**
* @return {@code true} if the column value is an empty clob function call, otherwise {@code false}.
*/
boolean isEmptyClobFunction();
/**
* @return {@code true} if the column value is an empty blob function call, otherwise {@code false}.
*/
boolean isEmptyBlobFunction();
/**
* This sets the database record value with the exception of LOBs
* @param columnData data

View File

@ -15,13 +15,14 @@
*/
public class LogMinerColumnValueImpl implements LogMinerColumnValue {
private static final String EMPTY_CLOB_FUNCTION = "EMPTY_CLOB()";
private static final String EMPTY_BLOB_FUNCTION = "EMPTY_BLOB()";
private String columnName;
private Object columnData;
private int columnType;
public LogMinerColumnValueImpl(String columnName, int columnType) {
public LogMinerColumnValueImpl(String columnName) {
this.columnName = columnName;
this.columnType = columnType;
}
@Override
@ -44,6 +45,20 @@ public void setColumnData(Object columnData) {
}
}
@Override
public boolean isEmptyClobFunction() {
return EMPTY_CLOB_FUNCTION.equals(columnData);
}
@Override
public boolean isEmptyBlobFunction() {
return EMPTY_BLOB_FUNCTION.equals(columnData);
}
public void setColumnName(String columnName) {
this.columnName = columnName;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -53,13 +68,17 @@ public boolean equals(Object o) {
return false;
}
LogMinerColumnValueImpl that = (LogMinerColumnValueImpl) o;
return columnType == that.columnType &&
Objects.equals(columnName, that.columnName) &&
return Objects.equals(columnName, that.columnName) &&
Objects.equals(columnData, that.columnData);
}
@Override
public int hashCode() {
return Objects.hash(columnName, columnData, columnType);
return Objects.hash(columnName, columnData);
}
@Override
public String toString() {
return "LogMinerColumnValueImpl{columnName=" + columnName + ", columnData=" + columnData + "}";
}
}

View File

@ -9,7 +9,6 @@
import java.util.List;
import io.debezium.connector.oracle.Scn;
import io.debezium.data.Envelope;
public interface LogMinerDmlEntry {
/**
@ -27,10 +26,9 @@ public interface LogMinerDmlEntry {
List<LogMinerColumnValue> getNewValues();
/**
* this getter
* @return Envelope.Operation enum
* @return LogMiner event operation type
*/
Envelope.Operation getCommandType();
int getOperation();
/**
* the scn obtained from a LogMiner entry.
@ -65,6 +63,11 @@ public interface LogMinerDmlEntry {
*/
String getRowId();
/**
* @return the sequence
*/
int getSequence();
/**
* sets scn obtained from a LogMiner entry
* @param scn it's value
@ -98,4 +101,9 @@ public interface LogMinerDmlEntry {
* @param rowId unique row identifier
*/
void setRowId(String rowId);
/**
* @param sequence operation sequence
*/
void setSequence(int sequence);
}

View File

@ -10,7 +10,6 @@
import java.util.Objects;
import io.debezium.connector.oracle.Scn;
import io.debezium.data.Envelope;
/**
* This class holds one parsed DML LogMiner record details
@ -18,25 +17,26 @@
*/
public class LogMinerDmlEntryImpl implements LogMinerDmlEntry {
private Envelope.Operation commandType;
private List<LogMinerColumnValue> newLmColumnValues;
private List<LogMinerColumnValue> oldLmColumnValues;
private final int operation;
private final List<LogMinerColumnValue> newLmColumnValues;
private final List<LogMinerColumnValue> oldLmColumnValues;
private String objectOwner;
private String objectName;
private Timestamp sourceTime;
private String transactionId;
private Scn scn;
private String rowId;
private int sequence;
public LogMinerDmlEntryImpl(Envelope.Operation commandType, List<LogMinerColumnValue> newLmColumnValues, List<LogMinerColumnValue> oldLmColumnValues) {
this.commandType = commandType;
public LogMinerDmlEntryImpl(int operation, List<LogMinerColumnValue> newLmColumnValues, List<LogMinerColumnValue> oldLmColumnValues) {
this.operation = operation;
this.newLmColumnValues = newLmColumnValues;
this.oldLmColumnValues = oldLmColumnValues;
}
@Override
public Envelope.Operation getCommandType() {
return commandType;
public int getOperation() {
return operation;
}
@Override
@ -74,6 +74,11 @@ public String getRowId() {
return rowId;
}
@Override
public int getSequence() {
return sequence;
}
@Override
public void setObjectName(String name) {
this.objectName = name;
@ -109,6 +114,11 @@ public void setRowId(String rowId) {
this.rowId = rowId;
}
@Override
public void setSequence(int sequence) {
this.sequence = sequence;
}
@Override
public boolean equals(Object o) {
if (this == o) {
@ -118,13 +128,18 @@ public boolean equals(Object o) {
return false;
}
LogMinerDmlEntryImpl that = (LogMinerDmlEntryImpl) o;
return commandType == that.commandType &&
return operation == that.operation &&
Objects.equals(newLmColumnValues, that.newLmColumnValues) &&
Objects.equals(oldLmColumnValues, that.oldLmColumnValues);
}
@Override
public int hashCode() {
return Objects.hash(commandType, newLmColumnValues, oldLmColumnValues);
return Objects.hash(operation, newLmColumnValues, oldLmColumnValues);
}
@Override
public String toString() {
return "{LogMinerDmlEntryImpl={operation=" + operation + ",newColumns=" + newLmColumnValues + ",oldColumns=" + oldLmColumnValues + "}";
}
}

View File

@ -8,6 +8,8 @@
import static org.fest.assertions.Assertions.assertThat;
import java.math.BigDecimal;
import java.sql.Clob;
import java.sql.NClob;
import java.sql.SQLException;
import java.time.LocalDate;
import java.time.LocalDateTime;
@ -22,10 +24,14 @@
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestRule;
import io.debezium.config.Configuration;
import io.debezium.config.Configuration.Builder;
import io.debezium.connector.oracle.junit.SkipTestDependingOnAdapterNameRule;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIs;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.SchemaAndValueField;
import io.debezium.data.VariableScaleDecimal;
@ -115,6 +121,17 @@ public abstract class AbstractOracleDatatypesTest extends AbstractConnectorTest
" primary key (id)" +
")";
private static final String DDL_CLOB = "create table debezium.type_clob (" +
" id numeric(9,0) not null, " +
" val_clob_inline clob, " +
" val_nclob_inline nclob, " +
" val_clob_short clob, " +
" val_nclob_short nclob, " +
" val_clob_long clob, " +
" val_nclob_long nclob, " +
" primary key (id)" +
")";
private static final List<SchemaAndValueField> EXPECTED_STRING = Arrays.asList(
new SchemaAndValueField("VAL_VARCHAR", Schema.OPTIONAL_STRING_SCHEMA, "v\u010d2"),
new SchemaAndValueField("VAL_VARCHAR2", Schema.OPTIONAL_STRING_SCHEMA, "v\u010d2"),
@ -216,20 +233,44 @@ public abstract class AbstractOracleDatatypesTest extends AbstractConnectorTest
new SchemaAndValueField("VAL_INT_YTM", MicroDuration.builder().optional().build(), -110451600_000_000L),
new SchemaAndValueField("VAL_INT_DTS", MicroDuration.builder().optional().build(), -93784_560_000L));
private static final String CLOB_JSON = Testing.Files.readResourceAsString("data/test_clob_data.json");
private static final String NCLOB_JSON = Testing.Files.readResourceAsString("data/test_clob_data2.json");
private static final List<SchemaAndValueField> EXPECTED_CLOB = Arrays.asList(
new SchemaAndValueField("VAL_CLOB_INLINE", Schema.OPTIONAL_STRING_SCHEMA, "TestClob123"),
new SchemaAndValueField("VAL_NCLOB_INLINE", Schema.OPTIONAL_STRING_SCHEMA, "TestNClob123"),
new SchemaAndValueField("VAL_CLOB_SHORT", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 0, 512)),
new SchemaAndValueField("VAL_NCLOB_SHORT", Schema.OPTIONAL_STRING_SCHEMA, part(NCLOB_JSON, 0, 512)),
new SchemaAndValueField("VAL_CLOB_LONG", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 0, 5000)),
new SchemaAndValueField("VAL_NCLOB_LONG", Schema.OPTIONAL_STRING_SCHEMA, part(NCLOB_JSON, 0, 5000)));
private static final List<SchemaAndValueField> EXPECTED_CLOB_UPDATE = Arrays.asList(
new SchemaAndValueField("VAL_CLOB_INLINE", Schema.OPTIONAL_STRING_SCHEMA, "TestClob123Update"),
new SchemaAndValueField("VAL_NCLOB_INLINE", Schema.OPTIONAL_STRING_SCHEMA, "TestNClob123Update"),
new SchemaAndValueField("VAL_CLOB_SHORT", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 1, 512)),
new SchemaAndValueField("VAL_NCLOB_SHORT", Schema.OPTIONAL_STRING_SCHEMA, part(NCLOB_JSON, 1, 512)),
new SchemaAndValueField("VAL_CLOB_LONG", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 1, 5000)),
new SchemaAndValueField("VAL_NCLOB_LONG", Schema.OPTIONAL_STRING_SCHEMA, part(NCLOB_JSON, 1, 5000)));
private static final String[] ALL_TABLES = {
"debezium.type_string",
"debezium.type_fp",
"debezium.type_int",
"debezium.type_time"
"debezium.type_time",
"debezium.type_clob"
};
private static final String[] ALL_DDLS = {
DDL_STRING,
DDL_FP,
DDL_INT,
DDL_TIME
DDL_TIME,
DDL_CLOB
};
@Rule
public final TestRule skipAdapterRule = new SkipTestDependingOnAdapterNameRule();
private static OracleConnection connection;
@BeforeClass
@ -551,6 +592,54 @@ public void timeTypesAsConnect() throws Exception {
assertRecord(after, EXPECTED_TIME_AS_CONNECT);
}
@Test
@SkipWhenAdapterNameIs(value = SkipWhenAdapterNameIs.AdapterName.XSTREAM, reason = "XStream doesn't yet support CLOB data types")
public void clobTypes() throws Exception {
int expectedRecordCount = 0;
if (insertRecordsDuringTest()) {
insertClobTypes();
}
Testing.debug("Inserted");
expectedRecordCount++;
SourceRecords records = consumeRecordsByTopic(expectedRecordCount);
List<SourceRecord> testTableRecords = records.recordsForTopic("server1.DEBEZIUM.TYPE_CLOB");
assertThat(testTableRecords).hasSize(expectedRecordCount);
SourceRecord record = testTableRecords.get(0);
VerifyRecord.isValid(record);
// insert
if (insertRecordsDuringTest()) {
VerifyRecord.isValidInsert(record, "ID", 1);
}
else {
VerifyRecord.isValidRead(record, "ID", 1);
}
Struct after = (Struct) ((Struct) record.value()).get("after");
assertRecord(after, EXPECTED_CLOB);
if (insertRecordsDuringTest()) {
// Update clob types
updateClobTypes();
records = consumeRecordsByTopic(1);
testTableRecords = records.recordsForTopic("server1.DEBEZIUM.TYPE_CLOB");
assertThat(testTableRecords).hasSize(1);
record = testTableRecords.get(0);
VerifyRecord.isValid(record);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = (Struct) ((Struct) record.value()).get("after");
assertRecord(after, EXPECTED_CLOB_UPDATE);
}
}
protected static void insertStringTypes() throws SQLException {
connection.execute("INSERT INTO debezium.type_string VALUES (1, 'v\u010d2', 'v\u010d2', 'nv\u010d2', 'c', 'n\u010d')");
connection.execute("COMMIT");
@ -583,6 +672,59 @@ protected static void insertTimeTypes() throws SQLException {
connection.execute("COMMIT");
}
protected static void insertClobTypes() throws SQLException {
Clob clob1 = connection.connection().createClob();
clob1.setString(1, part(CLOB_JSON, 0, 512));
Clob clob2 = connection.connection().createClob();
clob2.setString(1, part(CLOB_JSON, 0, 5000));
NClob nclob1 = connection.connection().createNClob();
nclob1.setString(1, part(NCLOB_JSON, 0, 512));
NClob nclob2 = connection.connection().createNClob();
nclob2.setString(1, part(NCLOB_JSON, 0, 5000));
connection.prepareQuery("INSERT INTO debezium.type_clob VALUES (1, ?, ?, ?, ?, ?, ?)", ps -> {
ps.setString(1, "TestClob123");
ps.setString(2, "TestNClob123");
ps.setClob(3, clob1);
ps.setNClob(4, nclob1);
ps.setClob(5, clob2);
ps.setNClob(6, nclob2);
}, null);
connection.commit();
}
protected static void updateClobTypes() throws Exception {
Clob clob1 = connection.connection().createClob();
clob1.setString(1, part(CLOB_JSON, 1, 512));
Clob clob2 = connection.connection().createClob();
clob2.setString(1, part(CLOB_JSON, 1, 5000));
NClob nclob1 = connection.connection().createNClob();
nclob1.setString(1, part(NCLOB_JSON, 1, 512));
NClob nclob2 = connection.connection().createNClob();
nclob2.setString(1, part(NCLOB_JSON, 1, 5000));
connection.prepareQuery("UPDATE debezium.type_clob SET VAL_CLOB_INLINE=?, VAL_NCLOB_INLINE=?, VAL_CLOB_SHORT=?, "
+ "VAL_NCLOB_SHORT=?, VAL_CLOB_LONG=?, VAL_NCLOB_LONG=? WHERE ID = 1", ps -> {
ps.setString(1, "TestClob123Update");
ps.setString(2, "TestNClob123Update");
ps.setClob(3, clob1);
ps.setNClob(4, nclob1);
ps.setClob(5, clob2);
ps.setNClob(6, nclob2);
}, null);
connection.commit();
}
private static String part(String text, int start, int length) {
return text == null ? "" : text.substring(start, Math.min(length, text.length()));
}
private void assertRecord(Struct record, List<SchemaAndValueField> expected) {
expected.forEach(schemaAndValueField -> schemaAndValueField.assertFor(record));
}

View File

@ -0,0 +1,1039 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import static org.fest.assertions.Assertions.assertThat;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Blob;
import java.sql.SQLException;
import java.time.Duration;
import java.util.Arrays;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.awaitility.Awaitility;
import org.fest.assertions.Fail;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestRule;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.junit.SkipTestDependingOnAdapterNameRule;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIs;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.data.VerifyRecord;
import io.debezium.doc.FixFor;
import io.debezium.embedded.AbstractConnectorTest;
import io.debezium.junit.logging.LogInterceptor;
import io.debezium.util.IoUtil;
import io.debezium.util.Testing;
/**
* Integration tests for BLOB data type support.
*
* @author Chris Cranford
*/
@SkipWhenAdapterNameIs(value = SkipWhenAdapterNameIs.AdapterName.XSTREAM, reason = "XStream does not yet support BLOB data types")
public class OracleBlobDataTypesIT extends AbstractConnectorTest {
private static final byte[] BIN_DATA = readBinaryData("data/test_lob_data.json");
@Rule
public final TestRule skipAdapterRule = new SkipTestDependingOnAdapterNameRule();
private OracleConnection connection;
@Before
public void before() {
connection = TestHelper.testConnection();
TestHelper.dropTable(connection, "BLOB_TEST");
setConsumeTimeout(TestHelper.defaultMessageConsumerPollTimeout(), TimeUnit.SECONDS);
initializeConnectorTestFramework();
Testing.Files.delete(TestHelper.DB_HISTORY_PATH);
}
@After
public void after() throws Exception {
if (connection != null) {
TestHelper.dropTable(connection, "BLOB_TEST");
connection.close();
}
}
@Test
@FixFor("DBZ-2948")
public void shouldSnapshotBlobDataTypes() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "primary key(id))";
connection.execute(ddl);
String dml = "INSERT INTO BLOB_TEST VALUES (1, utl_raw.cast_to_raw('Hello World'))";
connection.execute(dml);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidRead(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap("Hello World".getBytes(StandardCharsets.UTF_8)));
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamSmallBlobDataTypeValues() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1 = createBlob(part(BIN_DATA, 0, 100));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?)", p -> p.setBlob(1, blob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1.getBytes(1, 100)));
// Insert multiple records, same transaction
Blob blob2 = createBlob(part(BIN_DATA, 0, 200));
connection.prepareQuery("INSERT INTO debezium.blob_test values (2, ?)", p -> p.setBlob(1, blob2), null);
Blob blob3 = createBlob(part(BIN_DATA, 0, 300));
connection.prepareQuery("INSERT INTO debezium.blob_test values (3, ?)", p -> p.setBlob(1, blob3), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2.getBytes(1, 200)));
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3.getBytes(1, 300)));
// Update record
Blob blob1Update = createBlob(part(BIN_DATA, 1, 201));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 1", p -> p.setBlob(1, blob1Update), null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1Update.getBytes(1, 200)));
// Update multiple records, same transaction
Blob blob2Update = createBlob(part(BIN_DATA, 2, 202));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 2", p -> p.setBlob(1, blob2Update), null);
Blob blob3Update = createBlob(part(BIN_DATA, 3, 303));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 3", p -> p.setBlob(1, blob3Update), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2Update.getBytes(1, 200)));
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3Update.getBytes(1, 300)));
// Delete record
connection.execute("DELETE FROM debezium.blob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// blob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamSmallBlobDataTypeValuesWithNonBlobFields() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1 = createBlob(part(BIN_DATA, 0, 100));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?, 'Test1')", p -> p.setBlob(1, blob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1.getBytes(1, 100)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Insert multiple records, same transaction
Blob blob2 = createBlob(part(BIN_DATA, 0, 200));
connection.prepareQuery("INSERT INTO debezium.blob_test values (2, ?, 'Test2')", p -> p.setBlob(1, blob2), null);
Blob blob3 = createBlob(part(BIN_DATA, 0, 300));
connection.prepareQuery("INSERT INTO debezium.blob_test values (3, ?, 'Test3')", p -> p.setBlob(1, blob3), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2.getBytes(1, 200)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3.getBytes(1, 300)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3");
// Update record
Blob blob1Update = createBlob(part(BIN_DATA, 1, 201));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test1U' WHERE id = 1", p -> p.setBlob(1, blob1Update), null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1Update.getBytes(1, 200)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
// Update multiple records, same transaction
Blob blob2Update = createBlob(part(BIN_DATA, 2, 202));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test2U' WHERE id = 2", p -> p.setBlob(1, blob2Update), null);
Blob blob3Update = createBlob(part(BIN_DATA, 3, 303));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test3U' WHERE id = 3", p -> p.setBlob(1, blob3Update), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2Update.getBytes(1, 200)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3Update.getBytes(1, 300)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.blob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// blob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamLargeBlobDataTypeValues() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1 = createBlob(part(BIN_DATA, 0, 4000));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?)", p -> p.setBlob(1, blob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1.getBytes(1, 4000)));
// Insert multiple records, same transaction
Blob blob2 = createBlob(part(BIN_DATA, 10, 4010));
connection.prepareQuery("INSERT INTO debezium.blob_test values (2, ?)", p -> p.setBlob(1, blob2), null);
Blob blob3 = createBlob(part(BIN_DATA, 50, 4050));
connection.prepareQuery("INSERT INTO debezium.blob_test values (3, ?)", p -> p.setBlob(1, blob3), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2.getBytes(1, 4000)));
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3.getBytes(1, 4000)));
// Update record
Blob blob1Update = createBlob(part(BIN_DATA, 1, 4001));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 1", p -> p.setBlob(1, blob1Update), null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1Update.getBytes(1, 4000)));
// Update multiple records, same transaction
Blob blob2Update = createBlob(part(BIN_DATA, 2, 4002));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 2", p -> p.setBlob(1, blob2Update), null);
Blob blob3Update = createBlob(part(BIN_DATA, 3, 4003));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ? WHERE id = 3", p -> p.setBlob(1, blob3Update), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2Update.getBytes(1, 4000)));
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3Update.getBytes(1, 4000)));
// Delete record
connection.execute("DELETE FROM debezium.blob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// blob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(4);
records.forEach(System.out::println);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamLargeBlobDataTypeValuesWithNonBlobFields() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1 = createBlob(part(BIN_DATA, 0, 4000));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?, 'Test1')", p -> p.setBlob(1, blob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Insert multiple records, same transaction
Blob blob2 = createBlob(part(BIN_DATA, 10, 4010));
connection.prepareQuery("INSERT INTO debezium.blob_test values (2, ?, 'Test2')", p -> p.setBlob(1, blob2), null);
Blob blob3 = createBlob(part(BIN_DATA, 50, 4050));
connection.prepareQuery("INSERT INTO debezium.blob_test values (3, ?, 'Test3')", p -> p.setBlob(1, blob3), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3");
// Update record
Blob blob1Update = createBlob(part(BIN_DATA, 1, 4001));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test1U' WHERE id = 1", p -> p.setBlob(1, blob1Update), null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1Update.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
// Update multiple records, same transaction
Blob blob2Update = createBlob(part(BIN_DATA, 2, 4002));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test2U' WHERE id = 2", p -> p.setBlob(1, blob2Update), null);
Blob blob3Update = createBlob(part(BIN_DATA, 3, 4003));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blob = ?, val_data = 'Test3U' WHERE id = 3", p -> p.setBlob(1, blob3Update), null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2Update.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3Update.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.blob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// blob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(4);
records.forEach(System.out::println);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamMixedBlobDataTypeValuesWithNonBlobFieldsSameTable() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOBS blob, "
+ "VAL_BLOB blob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1a = createBlob(part(BIN_DATA, 1, 201));
Blob blob1b = createBlob(part(BIN_DATA, 0, 4000));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?, ?, 'Test1')", p -> {
p.setBlob(1, blob1a);
p.setBlob(2, blob1b);
}, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob1a.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1b.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Insert multiple records, same transaction
Blob blob2a = createBlob(part(BIN_DATA, 10, 210));
Blob blob2b = createBlob(part(BIN_DATA, 10, 4010));
connection.prepareQuery("INSERT INTO debezium.blob_test values (2, ?, ?, 'Test2')", p -> {
p.setBlob(1, blob2a);
p.setBlob(2, blob2b);
}, null);
Blob blob3a = createBlob(part(BIN_DATA, 50, 250));
Blob blob3b = createBlob(part(BIN_DATA, 50, 4050));
connection.prepareQuery("INSERT INTO debezium.blob_test values (3, ?, ?, 'Test3')", p -> {
p.setBlob(1, blob3a);
p.setBlob(2, blob3b);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob2a.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2b.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob3a.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3b.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3");
// Update record
Blob blob1aUpdate = createBlob(part(BIN_DATA, 5, 205));
Blob blob1bUpdate = createBlob(part(BIN_DATA, 1, 4001));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blobs = ?, val_blob = ?, val_data = 'Test1U' WHERE id = 1", p -> {
p.setBlob(1, blob1aUpdate);
p.setBlob(2, blob1bUpdate);
}, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob1aUpdate.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1bUpdate.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
// Update multiple records, same transaction
Blob blob2aUpdate = createBlob(part(BIN_DATA, 2, 202));
Blob blob2bUpdate = createBlob(part(BIN_DATA, 2, 4002));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blobs = ?, val_blob = ?, val_data = 'Test2U' WHERE id = 2", p -> {
p.setBlob(1, blob2aUpdate);
p.setBlob(2, blob2bUpdate);
}, null);
Blob blob3aUpdate = createBlob(part(BIN_DATA, 3, 203));
Blob blob3bUpdate = createBlob(part(BIN_DATA, 3, 4003));
connection.prepareQuery("UPDATE debezium.blob_test SET val_blobs = ?, val_blob = ?, val_data = 'Test3U' WHERE id = 3", p -> {
p.setBlob(1, blob3aUpdate);
p.setBlob(2, blob3bUpdate);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob2aUpdate.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob2bUpdate.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("BLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob3aUpdate.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob3bUpdate.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.blob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// blob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_BLOBS")).isNull();
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.blob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(4);
records.forEach(System.out::println);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_BLOBS")).isNull();
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// blob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_BLOBS")).isNull();
assertThat(before.get("VAL_BLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldNotStreamAnyChangesWhenLobEraseIsDetected() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOB blob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
LogInterceptor logInterceptor = new LogInterceptor();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1 = createBlob(part(BIN_DATA, 0, 4000));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?)", p -> p.setBlob(1, blob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1.getBytes(1, 4000)));
// Execute LOB_ERASE
connection.execute("DECLARE loc_b BLOB; amount integer; BEGIN "
+ "SELECT \"VAL_BLOB\" INTO loc_b FROM BLOB_TEST WHERE ID = 1 for update; "
+ "amount := 10;"
+ "dbms_lob.erase(loc_b, amount, 1); end;");
// Wait until the log has recorded the message.
Awaitility.await().atMost(Duration.ofMinutes(1)).until(() -> logInterceptor.containsWarnMessage("LOB_ERASE for table"));
assertNoRecordsToConsume();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamBlobFieldsWithPrimaryKeyChange() throws Exception {
String ddl = "CREATE TABLE BLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_BLOBS blob, "
+ "VAL_BLOB blob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.blob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.BLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Blob blob1a = createBlob(part(BIN_DATA, 1, 201));
Blob blob1b = createBlob(part(BIN_DATA, 0, 4000));
connection.prepareQuery("INSERT INTO debezium.blob_test values (1, ?, ?, 'Test1')", p -> {
p.setBlob(1, blob1a);
p.setBlob(2, blob1b);
}, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("BLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob1a.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1b.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Update record, including primary key
Blob blob1aUpdate = createBlob(part(BIN_DATA, 5, 205));
Blob blob1bUpdate = createBlob(part(BIN_DATA, 1, 4001));
connection.prepareQuery("UPDATE debezium.blob_test SET id = 2, val_blobs = ?, val_blob = ?, val_data = 'Test1U' WHERE id = 1", p -> {
p.setBlob(1, blob1aUpdate);
p.setBlob(2, blob1bUpdate);
}, null);
connection.commit();
// 3 records, one indicating the deletion of PK 1, tombstone, and PK 2
records = consumeRecordsByTopic(3);
assertThat(records.recordsForTopic(topicName("BLOB_TEST"))).hasSize(3);
record = records.recordsForTopic(topicName("BLOB_TEST")).get(2);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_BLOBS")).isEqualTo(ByteBuffer.wrap(blob1aUpdate.getBytes(1, 200)));
assertThat(after.get("VAL_BLOB")).isEqualTo(ByteBuffer.wrap(blob1bUpdate.getBytes(1, 4000)));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
}
private static byte[] part(byte[] buffer, int start, int length) {
return Arrays.copyOfRange(buffer, start, length);
}
private static Struct before(SourceRecord record) {
return ((Struct) record.value()).getStruct(Envelope.FieldName.BEFORE);
}
private static Struct after(SourceRecord record) {
return ((Struct) record.value()).getStruct(Envelope.FieldName.AFTER);
}
private static String topicName(String tableName) {
return TestHelper.SERVER_NAME + ".DEBEZIUM." + tableName;
}
private static byte[] readBinaryData(String pathOnClasspath) {
try (InputStream stream = Testing.Files.readResourceAsStream(pathOnClasspath)) {
return IoUtil.readBytes(stream);
}
catch (IOException e) {
Fail.fail("Unable to read '" + pathOnClasspath + "'", e);
return null;
}
}
private Blob createBlob(byte[] data) throws SQLException {
final Blob blob = connection.connection().createBlob();
blob.setBytes(1, data);
return blob;
}
}

View File

@ -0,0 +1,1370 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import static org.fest.assertions.Assertions.assertThat;
import java.sql.Clob;
import java.sql.NClob;
import java.sql.SQLException;
import java.time.Duration;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.awaitility.Awaitility;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestRule;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.junit.SkipTestDependingOnAdapterNameRule;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIs;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.data.VerifyRecord;
import io.debezium.doc.FixFor;
import io.debezium.embedded.AbstractConnectorTest;
import io.debezium.junit.logging.LogInterceptor;
import io.debezium.util.Testing;
/**
* Integration tests for CLOB data type support.
*
* @author Chris Cranford
*/
@SkipWhenAdapterNameIs(value = SkipWhenAdapterNameIs.AdapterName.XSTREAM, reason = "XStream does not yet support BLOB data types")
public class OracleClobDataTypeIT extends AbstractConnectorTest {
private static final String JSON_DATA = Testing.Files.readResourceAsString("data/test_lob_data.json");
private static final String JSON_DATA2 = Testing.Files.readResourceAsString("data/test_lob_data2.json");
@Rule
public final TestRule skipAdapterRule = new SkipTestDependingOnAdapterNameRule();
private OracleConnection connection;
@Before
public void before() {
connection = TestHelper.testConnection();
TestHelper.dropTable(connection, "CLOB_TEST");
setConsumeTimeout(TestHelper.defaultMessageConsumerPollTimeout(), TimeUnit.SECONDS);
initializeConnectorTestFramework();
Testing.Files.delete(TestHelper.DB_HISTORY_PATH);
}
@After
public void after() throws Exception {
if (connection != null) {
TestHelper.dropTable(connection, "CLOB_TEST");
connection.close();
}
}
@Test
@FixFor("DBZ-2948")
public void shouldSnapshotClobDataTypeValues() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB_SHORT clob, "
+ "VAL_CLOB_LONG clob, "
+ "VAL_NCLOB_SHORT nclob, "
+ "VAL_NCLOB_LONG nclob, "
+ "primary key(id))";
connection.execute(ddl);
Clob clob1 = createClob("Hello World");
Clob clob2 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob("Hello World");
NClob nclob2 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (1, ?, ?, ?, ?)", ps -> {
ps.setClob(1, clob1);
ps.setClob(2, clob2);
ps.setNClob(3, nclob1);
ps.setNClob(4, nclob2);
}, null);
connection.commit();
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidRead(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB_SHORT")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_CLOB_LONG")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB_SHORT")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_NCLOB_LONG")).isEqualTo(getClobString(nclob2));
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamInlineClobDataTypeValues() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 1000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 1000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (1, ?, ?)", ps -> { ps.setClob(1, clob1); ps.setNClob(2, nclob1); }, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 1000));
NClob nclob2 = createNClob(part(JSON_DATA2, 1, 1000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (2, ?, ?)", ps -> { ps.setClob(1, clob2); ps.setNClob(2, nclob2); }, null);
Clob clob3 = createClob(part(JSON_DATA, 2, 1000));
NClob nclob3 = createNClob(part(JSON_DATA2, 2, 1000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (3, ?, ?)", ps -> { ps.setClob(1, clob3); ps.setNClob(2, nclob3); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 1000));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 1000));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 1", ps -> { ps.setClob(1, clob1Update); ps.setNClob(2, nclob1Update); }, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 0, 1024));
NClob nclob2Update = createNClob(part(JSON_DATA2, 0, 1024));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 2", ps -> { ps.setClob(1, clob2Update); ps.setNClob(2, nclob2Update); }, null);
Clob clob3Update = createClob(part(JSON_DATA, 1, 1025));
NClob nclob3Update = createNClob(part(JSON_DATA2, 1, 1025));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 3", ps -> { ps.setClob(1, clob3Update); ps.setNClob(2, nclob3Update); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamInlineClobDataTypeValuesWithNonClobDataTypeField() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 1000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 1000));
connection.prepareQuery("INSERT INTO clob_test VALUES (1, ?, ?, 'Test1')", ps -> { ps.setClob(1, clob1); ps.setNClob(2, nclob1); }, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 1000));
NClob nclob2 = createNClob(part(JSON_DATA2, 1, 1000));
connection.prepareQuery("INSERT INTO clob_test VALUES (2, ?, ?, 'Test2')", ps -> { ps.setClob(1, clob2); ps.setNClob(2, nclob2); }, null);
Clob clob3 = createClob(part(JSON_DATA, 2, 1000));
NClob nclob3 = createNClob(part(JSON_DATA2, 2, 1000));
connection.prepareQuery("INSERT INTO clob_test VALUES (3, ?, ?, 'Test3')", ps -> { ps.setClob(1, clob3); ps.setNClob(2, nclob3); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3");
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 1000));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 1000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test1U' WHERE id = 1", ps -> {
ps.setClob(1, clob1Update);
ps.setNClob(2, nclob1Update);
}, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 0, 1024));
NClob nclob2Update = createNClob(part(JSON_DATA2, 0, 1024));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test2U' WHERE id = 2", ps -> {
ps.setClob(1, clob2Update);
ps.setNClob(2, nclob2Update);
}, null);
Clob clob3Update = createClob(part(JSON_DATA, 1, 1025));
NClob nclob3Update = createNClob(part(JSON_DATA2, 1, 1025));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test3U' WHERE id = 3", ps -> {
ps.setClob(1, clob3Update);
ps.setNClob(2, nclob3Update);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamLargeClobDataTypeValues() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (1, ?, ?)", ps -> { ps.setClob(1, clob1); ps.setNClob(2, nclob1); }, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 4450));
NClob nclob2 = createNClob(part(JSON_DATA2, 1, 4450));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (2, ?, ?)", ps -> { ps.setClob(1, clob2); ps.setNClob(2, nclob2); }, null);
Clob clob3 = createClob(part(JSON_DATA, 3, 4450));
NClob nclob3 = createNClob(part(JSON_DATA2, 3, 4450));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (3, ?, ?)", ps -> { ps.setClob(1, clob3); ps.setNClob(2, nclob3); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 4500));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 4500));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 1", ps -> { ps.setClob(1, clob1Update); ps.setNClob(2, nclob1Update); }, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 2, 5000));
NClob nclob2Update = createNClob(part(JSON_DATA2, 2, 5000));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 2", ps -> { ps.setClob(1, clob2Update); ps.setNClob(2, nclob2Update); }, null);
Clob clob3Update = createClob(part(JSON_DATA, 3, 5000));
NClob nclob3Update = createNClob(part(JSON_DATA2, 3, 5000));
connection.prepareQuery("UPDATE CLOB_TEST SET val_clob=?, val_nclob=? WHERE id = 3", ps -> { ps.setClob(1, clob3Update); ps.setNClob(2, nclob3Update); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
records.forEach(System.out::println);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamLargeClobDataTypeValuesWithNonClobDataTypeField() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "VAL_DATA varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO clob_test VALUES (1, ?, ?, 'Test1')", ps -> { ps.setClob(1, clob1); ps.setNClob(2, nclob1); }, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1");
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 4450));
NClob nclob2 = createNClob(part(JSON_DATA2, 2, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (2, ?, ?, 'Test2')", ps -> { ps.setClob(1, clob2); ps.setNClob(2, nclob2); }, null);
Clob clob3 = createClob(part(JSON_DATA, 3, 4450));
NClob nclob3 = createNClob(part(JSON_DATA2, 4, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (3, ?, ?, 'Test3')", ps -> { ps.setClob(1, clob3); ps.setNClob(2, nclob3); }, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3");
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 4500));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 4500));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test1U' WHERE id = 1", ps -> {
ps.setClob(1, clob1Update);
ps.setNClob(2, nclob1Update);
}, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test1U");
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 2, 5000));
NClob nclob2Update = createNClob(part(JSON_DATA2, 2, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test2U' WHERE id = 2", ps -> {
ps.setClob(1, clob2Update);
ps.setNClob(2, nclob2Update);
}, null);
Clob clob3Update = createClob(part(JSON_DATA, 3, 5000));
NClob nclob3Update = createNClob(part(JSON_DATA2, 3, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_data='Test3U' WHERE id = 3", ps -> {
ps.setClob(1, clob3Update);
ps.setNClob(2, nclob3Update);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
assertThat(after.get("VAL_DATA")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_DATA")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamMixedClobDataTypeValuesWithNonClobFieldsSameTable() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "VAL_CLOBS clob, "
+ "VAL_NCLOBS nclob, "
+ "VAL_VARCHAR2 varchar2(50),"
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO clob_test VALUES (1, ?, ?, 'ClobTest', 'NClobTest', 'Test1')", ps -> {
ps.setClob(1, clob1);
ps.setNClob(2, nclob1);
}, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1");
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 4450));
NClob nclob2 = createNClob(part(JSON_DATA2, 2, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (2, ?, ?, 'ClobTest2', 'NClobTest2', 'Test2')", ps -> {
ps.setClob(1, clob2);
ps.setNClob(2, nclob2);
}, null);
Clob clob3 = createClob(part(JSON_DATA, 3, 4450));
NClob nclob3 = createNClob(part(JSON_DATA2, 4, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (3, ?, ?, 'ClobTest3', 'NClobTest3', 'Test3')", ps -> {
ps.setClob(1, clob3);
ps.setNClob(2, nclob3);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest2");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest2");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest3");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest3");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test3");
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 4500));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 4500));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test1U' WHERE id = 1", ps -> {
ps.setClob(1, clob1Update);
ps.setNClob(2, nclob1Update);
ps.setString(3, "ClobTest1Updated");
ps.setString(4, "NClobTest1Updated");
}, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest1Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest1Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1U");
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 2, 5000));
NClob nclob2Update = createNClob(part(JSON_DATA2, 2, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test2U' WHERE id = 2", ps -> {
ps.setClob(1, clob2Update);
ps.setNClob(2, nclob2Update);
ps.setString(3, "ClobTest2Updated");
ps.setString(4, "NClobTest2Updated");
}, null);
Clob clob3Update = createClob(part(JSON_DATA, 3, 5000));
NClob nclob3Update = createNClob(part(JSON_DATA2, 3, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test3U' WHERE id = 3", ps -> {
ps.setClob(1, clob3Update);
ps.setNClob(2, nclob3Update);
ps.setString(3, "ClobTest3Updated");
ps.setString(4, "NClobTest3Updated");
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest2Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest2Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest3Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest3Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
@Test
@FixFor("DBZ-2948")
public void shouldNotStreamAnyChangesWhenLobEraseIsDetected() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
LogInterceptor logInterceptor = new LogInterceptor();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
connection.prepareQuery("INSERT INTO CLOB_TEST VALUES (1, ?)", ps -> ps.setClob(1, clob1), null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
// Execute LOB_ERASE
connection.execute("DECLARE loc_c CLOB; amount integer; BEGIN " +
"SELECT \"VAL_CLOB\" INTO loc_c FROM CLOB_TEST WHERE ID = 1 for update; " +
"amount := 10;" +
"dbms_lob.erase(loc_c, amount, 1); end;");
// Wait until the log has recorded the message.
Awaitility.await().atMost(Duration.ofMinutes(1)).until(() -> logInterceptor.containsWarnMessage("LOB_ERASE for table"));
assertNoRecordsToConsume();
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamClobDataTypeValuesWithPrimaryKeyChange() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "VAL_CLOBS clob, "
+ "VAL_NCLOBS nclob, "
+ "VAL_VARCHAR2 varchar2(50), "
+ "primary key(id))";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO clob_test VALUES (1, ?, ?, 'ClobTest', 'NClobTest', 'Test1')", ps -> {
ps.setClob(1, clob1);
ps.setNClob(2, nclob1);
}, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1");
// Update record, including primary key
Clob clob1Update = createClob(part(JSON_DATA, 1, 4500));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 4500));
connection.prepareQuery("UPDATE clob_test SET id=2, val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test1U' WHERE id = 1", ps -> {
ps.setClob(1, clob1Update);
ps.setNClob(2, nclob1Update);
ps.setString(3, "ClobTest1Updated");
ps.setString(4, "NClobTest1Updated");
}, null);
connection.commit();
// 3 records, one indicating the deletion of PK 1, tombstone, and PK 2
records = consumeRecordsByTopic(3);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(3);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest1Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest1Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1U");
}
@Test
@FixFor("DBZ-2948")
public void shouldStreamClobDataTypeValuesUsingBasicFileStorage() throws Exception {
String ddl = "CREATE TABLE CLOB_TEST ("
+ "ID numeric(9,0), "
+ "VAL_CLOB clob, "
+ "VAL_NCLOB nclob, "
+ "VAL_CLOBS clob, "
+ "VAL_NCLOBS nclob, "
+ "VAL_VARCHAR2 varchar2(50), "
+ "primary key(id)) "
+ "LOB(VAL_CLOB) STORE AS BASICFILE "
+ "LOB(VAL_NCLOB) STORE AS BASICFILE "
+ "LOB(VAL_CLOBS) STORE AS BASICFILE "
+ "LOB(VAL_NCLOBS) STORE AS BASICFILE";
connection.execute(ddl);
TestHelper.streamTable(connection, "debezium.clob_test");
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CLOB_TEST")
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Insert record
Clob clob1 = createClob(part(JSON_DATA, 0, 5000));
NClob nclob1 = createNClob(part(JSON_DATA2, 0, 5000));
connection.prepareQuery("INSERT INTO clob_test VALUES (1, ?, ?, 'ClobTest', 'NClobTest', 'Test1')", ps -> {
ps.setClob(1, clob1);
ps.setNClob(2, nclob1);
}, null);
connection.commit();
SourceRecords records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
SourceRecord record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1");
// Insert multiple records, same transaction
Clob clob2 = createClob(part(JSON_DATA, 1, 4450));
NClob nclob2 = createNClob(part(JSON_DATA2, 2, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (2, ?, ?, 'ClobTest2', 'NClobTest2', 'Test2')", ps -> {
ps.setClob(1, clob2);
ps.setNClob(2, nclob2);
}, null);
Clob clob3 = createClob(part(JSON_DATA, 3, 4450));
NClob nclob3 = createNClob(part(JSON_DATA2, 4, 4450));
connection.prepareQuery("INSERT INTO clob_test VALUES (3, ?, ?, 'ClobTest3', 'NClobTest3', 'Test3')", ps -> {
ps.setClob(1, clob3);
ps.setNClob(2, nclob3);
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidInsert(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest2");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest2");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test2");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidInsert(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest3");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest3");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test3");
// Update record
Clob clob1Update = createClob(part(JSON_DATA, 1, 4500));
NClob nclob1Update = createNClob(part(JSON_DATA2, 1, 4500));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test1U' WHERE id = 1", ps -> {
ps.setClob(1, clob1Update);
ps.setNClob(2, nclob1Update);
ps.setString(3, "ClobTest1Updated");
ps.setString(4, "NClobTest1Updated");
}, null);
connection.commit();
records = consumeRecordsByTopic(1);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(1);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 1);
after = after(record);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob1Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob1Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest1Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest1Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test1U");
// Update multiple records, same transaction
Clob clob2Update = createClob(part(JSON_DATA, 2, 5000));
NClob nclob2Update = createNClob(part(JSON_DATA2, 2, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test2U' WHERE id = 2", ps -> {
ps.setClob(1, clob2Update);
ps.setNClob(2, nclob2Update);
ps.setString(3, "ClobTest2Updated");
ps.setString(4, "NClobTest2Updated");
}, null);
Clob clob3Update = createClob(part(JSON_DATA, 3, 5000));
NClob nclob3Update = createNClob(part(JSON_DATA2, 3, 5000));
connection.prepareQuery("UPDATE clob_test SET val_clob=?, val_nclob=?, val_clobs=?, val_nclobs=?, val_varchar2='Test3U' WHERE id = 3", ps -> {
ps.setClob(1, clob3Update);
ps.setNClob(2, nclob3Update);
ps.setString(3, "ClobTest3Updated");
ps.setString(4, "NClobTest3Updated");
}, null);
connection.commit();
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidUpdate(record, "ID", 2);
after = after(record);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob2Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob2Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest2Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest2Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test2U");
record = records.recordsForTopic(topicName("CLOB_TEST")).get(1);
VerifyRecord.isValidUpdate(record, "ID", 3);
after = after(record);
assertThat(after.get("ID")).isEqualTo(3);
assertThat(after.get("VAL_CLOB")).isEqualTo(getClobString(clob3Update));
assertThat(after.get("VAL_NCLOB")).isEqualTo(getClobString(nclob3Update));
assertThat(after.get("VAL_CLOBS")).isEqualTo("ClobTest3Updated");
assertThat(after.get("VAL_NCLOBS")).isEqualTo("NClobTest3Updated");
assertThat(after.get("VAL_VARCHAR2")).isEqualTo("Test3U");
// Delete record
connection.execute("DELETE FROM debezium.clob_test WHERE id = 1");
// two records, delete + tombstone
records = consumeRecordsByTopic(2);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(2);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 1);
// clob fields will never have a "before" state
Struct before = before(record);
assertThat(before.get("ID")).isEqualTo(1);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test1U");
assertThat(after(record)).isNull();
// Delete multiple records, same transaction
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 2");
connection.executeWithoutCommitting("DELETE FROM debezium.clob_test WHERE id = 3");
connection.execute("COMMIT");
// 2 deletes + 2 tombstones
records = consumeRecordsByTopic(4);
assertThat(records.recordsForTopic(topicName("CLOB_TEST"))).hasSize(4);
record = records.recordsForTopic(topicName("CLOB_TEST")).get(0);
VerifyRecord.isValidDelete(record, "ID", 2);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(2);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test2U");
assertThat(after(record)).isNull();
record = records.recordsForTopic(topicName("CLOB_TEST")).get(2);
VerifyRecord.isValidDelete(record, "ID", 3);
// clob fields will never have a "before" state
before = before(record);
assertThat(before.get("ID")).isEqualTo(3);
assertThat(before.get("VAL_CLOB")).isNull();
assertThat(before.get("VAL_NCLOB")).isNull();
assertThat(before.get("VAL_VARCHAR2")).isEqualTo("Test3U");
assertThat(after(record)).isNull();
}
private Clob createClob(String data) throws SQLException {
Clob clob = connection.connection().createClob();
clob.setString(1, data);
return clob;
}
private NClob createNClob(String data) throws SQLException {
NClob nclob = connection.connection().createNClob();
nclob.setString(1, data);
return nclob;
}
private static String part(String text, int start, int length) {
return text == null ? "" : text.substring(start, Math.min(length, text.length()));
}
private static Struct before(SourceRecord record) {
return ((Struct) record.value()).getStruct(Envelope.FieldName.BEFORE);
}
private static Struct after(SourceRecord record) {
return ((Struct) record.value()).getStruct(Envelope.FieldName.AFTER);
}
private static String topicName(String tableName) {
return TestHelper.SERVER_NAME + ".DEBEZIUM." + tableName;
}
private static String getClobString(Clob clob) throws SQLException {
return clob.getSubString(1, (int) clob.length());
}
}

View File

@ -1573,6 +1573,7 @@ record = records.recordsForTopic("server1.DEBEZIUM.TEST_IOT").get(0);
}
}
// todo: should this test be removed since its now covered in OracleClobDataTypesIT?
@Test
@FixFor("DBZ-3257")
@SkipWhenAdapterNameIsNot(value = SkipWhenAdapterNameIsNot.AdapterName.LOGMINER)

View File

@ -38,6 +38,7 @@ public static void beforeClass() throws SQLException {
insertFpTypes();
insertIntTypes();
insertTimeTypes();
insertClobTypes();
}
@Before
@ -81,6 +82,8 @@ private String getTableIncludeList() {
case "timeTypesAsAdaptiveMicroseconds":
case "timeTypesAsConnect":
return "debezium.type_time";
case "clobTypes":
return "debezium.type_clob";
default:
throw new IllegalArgumentException("Unexpected test method: " + name.getMethodName());
}

View File

@ -12,7 +12,6 @@
import io.debezium.connector.oracle.logminer.parser.LogMinerDmlParser;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.data.Envelope.Operation;
import io.debezium.doc.FixFor;
import io.debezium.relational.Column;
import io.debezium.relational.Table;
@ -56,7 +55,7 @@ public void testParsingInsert() throws Exception {
"TO_DATE('2020-02-01 00:00:00', 'YYYY-MM-DD HH24:MI:SS'),Unsupported Type,NULL,NULL);";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.CREATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.INSERT);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(9);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("ID");
@ -104,7 +103,7 @@ public void testParsingUpdate() throws Exception {
"\"UT2\" = Unsupported Type and \"C1\" = NULL and \"IS\" IS NULL and \"IS2\" IS NULL;";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.UPDATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.UPDATE);
assertThat(entry.getOldValues()).hasSize(10);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("ID");
assertThat(entry.getOldValues().get(1).getColumnName()).isEqualTo("NAME");
@ -170,7 +169,7 @@ public void testParsingDelete() throws Exception {
"\"IS\" IS NULL and \"IS2\" IS NULL;";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.DELETE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.DELETE);
assertThat(entry.getOldValues()).hasSize(8);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("ID");
assertThat(entry.getOldValues().get(1).getColumnName()).isEqualTo("NAME");
@ -205,7 +204,7 @@ public void testParsingUpdateWithNoWhereClauseIsAcceptable() throws Exception {
String sql = "update \"DEBEZIUM\".\"TEST\" set \"COL1\" = '1', \"COL2\" = NULL, \"COL3\" = 'Hello';";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.UPDATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.UPDATE);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(4);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("COL1");
@ -232,7 +231,7 @@ public void testParsingDeleteWithNoWhereClauseIsAcceptable() throws Exception {
String sql = "delete from \"DEBEZIUM\".\"TEST\";";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.DELETE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.DELETE);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).isEmpty();
}
@ -248,7 +247,7 @@ public void testNameWithWhitespaces() throws Exception {
String sql = "insert into \"UNKNOWN\".\"OBJ# 74858\"(\"COL 1\") values (1)";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.CREATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.INSERT);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(1);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("COL 1");
@ -270,7 +269,7 @@ public void testParsingUpdateWithNoWhereClauseFunctionAsLastColumn() throws Exce
"\"PAY_STATUS\" = '10111015', \"IS_DEL\" = '0', \"TM_UPDATE\" = TO_DATE('2021-03-17 10:18:55', 'YYYY-MM-DD HH24:MI:SS');";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.UPDATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.UPDATE);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(5);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("AMOUNT_PAID");
@ -297,7 +296,7 @@ public void shouldParsingRedoSqlWithParenthesisInFunctionArgumentStrings() throw
String sql = "insert into \"DEBEZIUM\".\"TEST\" (\"C1\", \"C2\") values (UNISTR('\\963F\\72F8\\5C0F\\706B\\8F66\\5BB6\\5EAD\\7968(\\60CA\\559C\\FF09\\FF082161\\FF09'), NULL);";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.CREATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.INSERT);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(2);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("C1");
@ -310,7 +309,7 @@ public void shouldParsingRedoSqlWithParenthesisInFunctionArgumentStrings() throw
"\"C2\" = UNISTR('\\963F\\72F8\\5C0F\\706B\\8F66\\5BB6\\5EAD\\7968(\\60CA\\559C\\FF09\\FF082161\\FF09') " +
"where \"C1\" = UNISTR('\\963F\\72F8\\5C0F\\706B\\8F66\\5BB6\\5EAD\\7968(\\60CA\\559C\\FF09\\FF082161\\FF09');";
entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.UPDATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.UPDATE);
assertThat(entry.getOldValues()).hasSize(2);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("C1");
assertThat(entry.getOldValues().get(0).getColumnData())
@ -327,7 +326,7 @@ public void shouldParsingRedoSqlWithParenthesisInFunctionArgumentStrings() throw
sql = "delete from \"DEBEZIUM\".\"TEST\" where \"C1\" = UNISTR('\\963F\\72F8\\5C0F\\706B\\8F66\\5BB6\\5EAD\\7968(\\60CA\\559C\\FF09\\FF082161\\FF09');";
entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.DELETE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.DELETE);
assertThat(entry.getOldValues()).hasSize(2);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("C1");
assertThat(entry.getOldValues().get(0).getColumnData())
@ -348,7 +347,7 @@ public void testParsingDoubleSingleQuoteInWhereClause() throws Exception {
String sql = "insert into \"DEBEZIUM\".\"TEST\"(\"COL1\",\"COL2\") values ('Bob''s dog','0');";
LogMinerDmlEntry entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.CREATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.INSERT);
assertThat(entry.getOldValues()).isEmpty();
assertThat(entry.getNewValues()).hasSize(2);
assertThat(entry.getNewValues().get(0).getColumnName()).isEqualTo("COL1");
@ -358,7 +357,7 @@ public void testParsingDoubleSingleQuoteInWhereClause() throws Exception {
sql = "update \"DEBEZIUM\".\"TEST\" set \"COL2\" = '1' where \"COL1\" = 'Bob''s dog' and \"COL2\" = '0';";
entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.UPDATE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.UPDATE);
assertThat(entry.getOldValues()).hasSize(2);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("COL1");
assertThat(entry.getOldValues().get(0).getColumnData()).isEqualTo("Bob''s dog");
@ -372,7 +371,7 @@ public void testParsingDoubleSingleQuoteInWhereClause() throws Exception {
sql = "delete from \"DEBEZIUM\".\"TEST\" where \"COL1\" = 'Bob''s dog' and \"COL2\" = '1';";
entry = fastDmlParser.parse(sql, table, null);
assertThat(entry.getCommandType()).isEqualTo(Operation.DELETE);
assertThat(entry.getOperation()).isEqualTo(RowMapper.DELETE);
assertThat(entry.getOldValues()).hasSize(2);
assertThat(entry.getOldValues().get(0).getColumnName()).isEqualTo("COL1");
assertThat(entry.getOldValues().get(0).getColumnData()).isEqualTo("Bob''s dog");

View File

@ -36,7 +36,6 @@
import io.debezium.connector.oracle.logminer.valueholder.LogMinerColumnValue;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.doc.FixFor;
import io.debezium.relational.Table;
import io.debezium.relational.TableId;
@ -275,7 +274,7 @@ public void shouldParseUpdateNoChangesTable() throws Exception {
"and COL8 = TO_TIMESTAMP('2019-05-14 02:28:32') and col11 = " + SPATIAL_DATA + ";";
LogMinerDmlEntry record = sqlDmlParser.parse(dml, tables.forTable(TABLE_ID), "");
boolean pass = record.getCommandType().equals(Envelope.Operation.UPDATE)
boolean pass = record.getOperation() == RowMapper.UPDATE
&& record.getOldValues().size() == record.getNewValues().size()
&& record.getNewValues().containsAll(record.getOldValues());
assertThat(pass);
@ -355,7 +354,7 @@ private void assertDmlParserException(String sql, DmlParser parser, Table table,
private void verifyUpdate(LogMinerDmlEntry record, boolean checkGeometry, boolean checkOldValues, int oldValuesNumber) {
// validate
assertThat(record.getCommandType()).isEqualTo(Envelope.Operation.UPDATE);
assertThat(record.getOperation()).isEqualTo(RowMapper.UPDATE);
List<LogMinerColumnValue> newValues = record.getNewValues();
assertThat(newValues.size()).isEqualTo(14);
String concatenatedNames = newValues.stream().map(LogMinerColumnValue::getColumnName).collect(Collectors.joining());
@ -462,7 +461,7 @@ private void verifyInsert(LogMinerDmlEntry record) {
List<LogMinerColumnValue> oldValues = record.getOldValues();
assertThat(oldValues.size()).isEqualTo(0);
assertThat(record.getCommandType()).isEqualTo(Envelope.Operation.CREATE);
assertThat(record.getOperation()).isEqualTo(RowMapper.INSERT);
List<LogMinerColumnValue> newValues = record.getNewValues();
assertThat(newValues.size()).isEqualTo(14);
@ -483,7 +482,7 @@ private void verifyInsert(LogMinerDmlEntry record) {
}
private void verifyDelete(LogMinerDmlEntry record, boolean checkOldValues) {
assertThat(record.getCommandType()).isEqualTo(Envelope.Operation.DELETE);
assertThat(record.getOperation()).isEqualTo(RowMapper.DELETE);
List<LogMinerColumnValue> newValues = record.getNewValues();
assertThat(newValues.size()).isEqualTo(0);

View File

@ -33,10 +33,11 @@ public class SqlUtilsTest {
public TestRule skipRule = new SkipTestDependingOnAdapterNameRule();
private static final String LOG_MINER_CONTENT_QUERY_TEMPLATE = "SELECT SCN, SQL_REDO, OPERATION_CODE, TIMESTAMP, " +
"XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ROW_ID, ROLLBACK " +
"XID, CSF, TABLE_NAME, SEG_OWNER, OPERATION, USERNAME, ROW_ID, ROLLBACK, SEQUENCE#, RS_ID, " +
"ORA_HASH(SCN||OPERATION||RS_ID||SEQUENCE#||RTRIM(SUBSTR(SQL_REDO,1,256))) " +
"FROM V$LOGMNR_CONTENTS WHERE SCN > ? AND SCN <= ? AND ((" +
"OPERATION_CODE IN (5,34) AND USERNAME NOT IN ('SYS','SYSTEM','${user}') AND INFO NOT LIKE 'INTERNAL DDL%') " +
"OR (OPERATION_CODE IN (7,36)) " +
"OPERATION_CODE IN (5,9,10,11,29,34) AND USERNAME NOT IN ('SYS','SYSTEM','${user}') AND INFO NOT LIKE 'INTERNAL DDL%') " +
"OR (OPERATION_CODE IN (6,7,36)) " +
"OR (OPERATION_CODE IN (1,2,3) " +
"AND TABLE_NAME != '" + SqlUtils.LOGMNR_FLUSH_TABLE + "' " +
"${systemTablePredicate}" +

View File

@ -40,7 +40,6 @@
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntryImpl;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
@ -65,7 +64,7 @@ public class TransactionalBufferTest {
private static final TableId TABLE_ID = new TableId(TestHelper.SERVER_NAME, "DEBEZIUM", "TEST");
private static final String ROW_ID = "AAABCD871DFAA";
private static final String OTHER_ROW_ID = "BAABCD871DFAA";
private static final LogMinerDmlEntry DML_ENTRY = new LogMinerDmlEntryImpl(Envelope.Operation.CREATE, Collections.emptyList(), Collections.emptyList());
private static final LogMinerDmlEntry DML_ENTRY = new LogMinerDmlEntryImpl(RowMapper.INSERT, Collections.emptyList(), Collections.emptyList());
private static final Configuration config = new Configuration() {
@Override
@ -227,6 +226,6 @@ public void testTransactionDump() {
}
private void registerDmlOperation(String txId, Scn scn, String rowId) {
transactionalBuffer.registerDmlOperation(RowMapper.INSERT, txId, scn, TABLE_ID, DML_ENTRY, Instant.now(), rowId);
transactionalBuffer.registerDmlOperation(RowMapper.INSERT, txId, scn, TABLE_ID, DML_ENTRY, Instant.now(), rowId, null, 0L);
}
}

View File

@ -34,7 +34,6 @@
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntry;
import io.debezium.connector.oracle.logminer.valueholder.LogMinerDmlEntryImpl;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.relational.TableId;
import io.debezium.relational.Tables;
import io.debezium.util.IoUtil;
@ -66,18 +65,18 @@ public void setUp() {
@Test
public void testValueHolders() throws Exception {
LogMinerColumnValue column1 = new LogMinerColumnValueImpl("COLUMN1", Types.NUMERIC);
LogMinerColumnValue column1 = new LogMinerColumnValueImpl("COLUMN1");
assertThat(column1.equals(column1)).isTrue();
assertThat(column1.equals(null)).isFalse();
assertThat(new LogMinerColumnValueWrapper(column1).isProcessed()).isFalse();
column1.setColumnData(new BigDecimal(5));
LogMinerColumnValue column2 = new LogMinerColumnValueImpl("COLUMN2", Types.VARCHAR);
LogMinerColumnValue column2 = new LogMinerColumnValueImpl("COLUMN2");
column2.setColumnData("Text");
List<LogMinerColumnValue> newValues = new ArrayList<>();
newValues.add(column1);
newValues.add(column2);
LogMinerDmlEntryImpl dmlEntryExpected = new LogMinerDmlEntryImpl(Envelope.Operation.CREATE, newValues, Collections.emptyList());
LogMinerDmlEntryImpl dmlEntryExpected = new LogMinerDmlEntryImpl(RowMapper.INSERT, newValues, Collections.emptyList());
dmlEntryExpected.setTransactionId("transaction_id");
dmlEntryExpected.setObjectName(TABLE_NAME);
dmlEntryExpected.setObjectOwner(SCHEMA_NAME);
@ -91,7 +90,7 @@ public void testValueHolders() throws Exception {
LogMinerDmlEntry dmlEntryParsed = sqlDmlParser.parse(dml, tables.forTable(TABLE_ID), "1");
assertThat(dmlEntryParsed.equals(dmlEntryExpected)).isTrue();
assertThat(dmlEntryExpected.getCommandType() == Envelope.Operation.CREATE).isTrue();
assertThat(dmlEntryExpected.getOperation()).isEqualTo(RowMapper.INSERT);
assertThat(dmlEntryExpected.getScn().equals(SCN_ONE)).isTrue();
assertThat(dmlEntryExpected.getSourceTime().equals(new Timestamp(1000))).isTrue();
assertThat(dmlEntryExpected.getTransactionId().equals("transaction_id")).isTrue();

View File

@ -0,0 +1,2339 @@
[
{
"_id": "6081cb8b0665b19449a392c8",
"index": 0,
"guid": "bc60b7bf-db6e-4dc8-9703-a451523757b1",
"isActive": true,
"balance": "$3,706.43",
"picture": "http://placehold.it/32x32",
"age": 33,
"eyeColor": "green",
"name": {
"first": "Mccullough",
"last": "Church"
},
"company": "ZOGAK",
"email": "mccullough.church@zogak.com",
"phone": "+1 (847) 519-3021",
"address": "962 Varet Street, Helen, Iowa, 1780",
"about": "Aliqua laborum sunt duis sit culpa consectetur irure proident veniam consequat culpa adipisicing aute qui. Esse aliquip culpa adipisicing non sunt pariatur cupidatat nostrud excepteur id voluptate magna. Nulla laboris voluptate velit aute est Lorem sit consequat commodo. Anim magna eiusmod sunt dolore officia proident cupidatat elit aute adipisicing officia consectetur amet Lorem. Laboris est labore aute magna amet fugiat voluptate labore. Dolor eiusmod magna qui commodo ullamco. Laboris ea velit anim dolore consectetur aute sit.",
"registered": "Monday, April 11, 2016 12:03 PM",
"latitude": "87.61937",
"longitude": "-21.610011",
"tags": [
"eu",
"est",
"aliqua",
"nostrud",
"nisi"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Pittman Carroll"
},
{
"id": 1,
"name": "Bernadine Salazar"
},
{
"id": 2,
"name": "Misty Alexander"
}
],
"greeting": "Hello, Mccullough! You have 9 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b3fe5da60ff3fbe26",
"index": 1,
"guid": "c25d3af0-fd20-47b4-9999-0dfb746c4050",
"isActive": true,
"balance": "$3,025.93",
"picture": "http://placehold.it/32x32",
"age": 37,
"eyeColor": "green",
"name": {
"first": "Becker",
"last": "Aguilar"
},
"company": "EPLOSION",
"email": "becker.aguilar@eplosion.biz",
"phone": "+1 (956) 543-2662",
"address": "533 Tapscott Street, Camptown, Arkansas, 9088",
"about": "Anim anim deserunt consequat adipisicing sunt et. Mollit ex ut consectetur irure velit enim eiusmod elit et esse nisi. Magna aliqua ullamco elit tempor eu fugiat sit nulla. Et deserunt mollit occaecat minim aliquip.",
"registered": "Wednesday, October 16, 2019 8:48 AM",
"latitude": "-83.80244",
"longitude": "-73.061613",
"tags": [
"ad",
"et",
"cupidatat",
"magna",
"occaecat"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Stephens Cain"
},
{
"id": 1,
"name": "Marquita Ewing"
},
{
"id": 2,
"name": "Parker Padilla"
}
],
"greeting": "Hello, Becker! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8bb2fa88cf9e55cf4c",
"index": 2,
"guid": "f80f62d2-28ab-443d-abf2-fdcf0d77bdd5",
"isActive": false,
"balance": "$2,568.00",
"picture": "http://placehold.it/32x32",
"age": 30,
"eyeColor": "green",
"name": {
"first": "Carter",
"last": "Fletcher"
},
"company": "GRAINSPOT",
"email": "carter.fletcher@grainspot.biz",
"phone": "+1 (925) 460-2087",
"address": "157 Hoyts Lane, Lindcove, Nevada, 3328",
"about": "Aliquip irure esse minim officia non eiusmod veniam culpa ipsum quis dolore aute. Sunt nisi elit proident aute velit proident sunt excepteur consequat aliqua. Qui duis eu sint est nostrud aute consectetur. Ullamco laboris sint tempor ullamco tempor aliqua laborum ullamco do.",
"registered": "Monday, August 14, 2017 8:43 AM",
"latitude": "-42.276593",
"longitude": "-97.520934",
"tags": [
"ea",
"consequat",
"ea",
"ea",
"laborum"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Cathleen Acosta"
},
{
"id": 1,
"name": "Marilyn Townsend"
},
{
"id": 2,
"name": "Fletcher Hines"
}
],
"greeting": "Hello, Carter! You have 6 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b902c09516716d08f",
"index": 3,
"guid": "13e2f920-cf7c-4f13-818f-a63e359d7ffe",
"isActive": false,
"balance": "$3,594.25",
"picture": "http://placehold.it/32x32",
"age": 39,
"eyeColor": "brown",
"name": {
"first": "Jayne",
"last": "Woodard"
},
"company": "AMRIL",
"email": "jayne.woodard@amril.ca",
"phone": "+1 (843) 497-2710",
"address": "636 Sands Street, Kilbourne, Ohio, 3036",
"about": "In fugiat non consequat cupidatat esse in qui consectetur pariatur officia qui irure labore veniam. Dolore laboris in laboris quis cupidatat mollit velit. Labore laboris eiusmod consequat tempor occaecat consectetur. Laborum et consequat tempor enim enim voluptate aliquip magna qui incididunt incididunt mollit. Mollit ipsum exercitation minim nulla eu irure eiusmod consequat duis ullamco est excepteur cillum cupidatat. Reprehenderit commodo magna magna consectetur quis laboris labore dolor fugiat eu. Officia velit esse amet reprehenderit Lorem excepteur incididunt aliqua exercitation laborum sunt consectetur laborum labore.",
"registered": "Sunday, December 16, 2018 7:11 PM",
"latitude": "74.441768",
"longitude": "70.387747",
"tags": [
"duis",
"cillum",
"dolore",
"sint",
"aliqua"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Marcy Hernandez"
},
{
"id": 1,
"name": "Bessie Gonzales"
},
{
"id": 2,
"name": "Catalina Hoover"
}
],
"greeting": "Hello, Jayne! You have 7 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8bbf29df52eafa1db2",
"index": 4,
"guid": "8f20052c-0bdf-4d06-8e91-41be93be31a4",
"isActive": true,
"balance": "$2,988.33",
"picture": "http://placehold.it/32x32",
"age": 39,
"eyeColor": "brown",
"name": {
"first": "Camille",
"last": "Cote"
},
"company": "DEVILTOE",
"email": "camille.cote@deviltoe.tv",
"phone": "+1 (889) 470-2770",
"address": "593 McKinley Avenue, Fivepointville, North Carolina, 3750",
"about": "Ad commodo magna exercitation elit voluptate voluptate nulla ea. Nisi magna adipisicing dolore exercitation cillum quis quis elit. Dolore voluptate quis non ut magna quis minim duis do. Duis est excepteur ut et irure aliquip magna mollit aliquip qui duis cillum laboris aliquip.",
"registered": "Saturday, February 24, 2018 6:06 AM",
"latitude": "-9.823305",
"longitude": "148.116816",
"tags": [
"eu",
"mollit",
"amet",
"ex",
"irure"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Tanner Faulkner"
},
{
"id": 1,
"name": "Hickman Moon"
},
{
"id": 2,
"name": "Harriett Joyner"
}
],
"greeting": "Hello, Camille! You have 5 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b69121a8dc791bbe8",
"index": 5,
"guid": "00c470af-0961-4a28-8ff1-b84fef0afea1",
"isActive": true,
"balance": "$3,608.43",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "brown",
"name": {
"first": "Flynn",
"last": "Hahn"
},
"company": "INTERODEO",
"email": "flynn.hahn@interodeo.name",
"phone": "+1 (807) 537-2898",
"address": "133 Jerome Avenue, Tioga, Utah, 5576",
"about": "Reprehenderit sint amet tempor culpa aute officia non occaecat commodo qui qui ad dolore ut. Minim magna mollit in velit culpa exercitation qui cillum tempor proident magna tempor excepteur consectetur. Dolore qui et elit mollit minim elit veniam eiusmod. Fugiat deserunt duis aliquip eiusmod laborum ut.",
"registered": "Sunday, September 20, 2020 11:01 AM",
"latitude": "30.320746",
"longitude": "-115.363509",
"tags": [
"qui",
"labore",
"eu",
"sint",
"non"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Mcfadden Palmer"
},
{
"id": 1,
"name": "Bethany Cruz"
},
{
"id": 2,
"name": "Fields Neal"
}
],
"greeting": "Hello, Flynn! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b2d328f8c363091c0",
"index": 6,
"guid": "b521aa0d-f686-4132-8d50-d82ae76517ac",
"isActive": true,
"balance": "$2,752.92",
"picture": "http://placehold.it/32x32",
"age": 40,
"eyeColor": "brown",
"name": {
"first": "Merritt",
"last": "Odom"
},
"company": "RAMJOB",
"email": "merritt.odom@ramjob.me",
"phone": "+1 (845) 593-3891",
"address": "679 Ridge Boulevard, Harborton, Nebraska, 8934",
"about": "Exercitation minim cupidatat sunt excepteur exercitation aute. Magna dolore id cillum sunt aliquip dolore et duis dolor aliquip enim eu. Nostrud culpa commodo aute non anim nulla. Mollit sit ipsum irure fugiat laborum culpa quis nisi consequat. Occaecat ipsum ea ad mollit incididunt veniam ad esse nostrud. Nulla sint ex occaecat ullamco laborum laborum adipisicing minim cupidatat non culpa enim eu minim.",
"registered": "Wednesday, November 18, 2020 2:05 AM",
"latitude": "13.062866",
"longitude": "-178.474174",
"tags": [
"qui",
"ea",
"laboris",
"cillum",
"ullamco"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Wooten Eaton"
},
{
"id": 1,
"name": "Frances England"
},
{
"id": 2,
"name": "Santiago Hammond"
}
],
"greeting": "Hello, Merritt! You have 6 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b5b716d4d0704fe88",
"index": 7,
"guid": "106804eb-1670-4550-8e24-b319b16468ee",
"isActive": true,
"balance": "$1,086.71",
"picture": "http://placehold.it/32x32",
"age": 22,
"eyeColor": "brown",
"name": {
"first": "Whitney",
"last": "Barton"
},
"company": "SLOGANAUT",
"email": "whitney.barton@sloganaut.org",
"phone": "+1 (942) 536-3135",
"address": "497 Monroe Street, Hegins, Northern Mariana Islands, 4048",
"about": "Nisi commodo velit nulla Lorem nulla sint id excepteur ea consequat. Enim culpa officia proident quis cupidatat pariatur. Sunt pariatur ullamco culpa officia excepteur adipisicing proident eiusmod culpa occaecat. Aliquip nostrud enim eu occaecat.",
"registered": "Monday, September 30, 2019 6:30 AM",
"latitude": "-54.99807",
"longitude": "-95.05969",
"tags": [
"eiusmod",
"excepteur",
"reprehenderit",
"sint",
"elit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Mcbride Bray"
},
{
"id": 1,
"name": "Raquel Jenkins"
},
{
"id": 2,
"name": "Conrad Stuart"
}
],
"greeting": "Hello, Whitney! You have 10 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b46d0b7d38756a669",
"index": 8,
"guid": "2adee5b8-e316-4d04-9a36-c2e47a48d181",
"isActive": false,
"balance": "$3,626.20",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "green",
"name": {
"first": "Colon",
"last": "Allen"
},
"company": "SURETECH",
"email": "colon.allen@suretech.io",
"phone": "+1 (868) 570-3328",
"address": "993 Hart Street, Cawood, Colorado, 6918",
"about": "Est et laboris elit nisi ut culpa ullamco duis aute ea culpa occaecat. Qui cupidatat laboris nostrud exercitation laboris proident ad sit proident ut id sint. Occaecat laborum ipsum ullamco esse ex duis officia magna sit laborum ipsum quis. Minim irure qui commodo adipisicing incididunt reprehenderit aute. Consequat cupidatat in duis sit proident eu consequat ullamco in commodo fugiat irure consequat aliqua.",
"registered": "Monday, December 1, 2014 8:22 AM",
"latitude": "-61.151252",
"longitude": "144.245",
"tags": [
"officia",
"consequat",
"consequat",
"irure",
"mollit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Jewel Hall"
},
{
"id": 1,
"name": "Emerson Obrien"
},
{
"id": 2,
"name": "Acosta Bauer"
}
],
"greeting": "Hello, Colon! You have 7 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b651f0846c573fbef",
"index": 9,
"guid": "fe880faf-615d-4488-8068-8963fe47606d",
"isActive": false,
"balance": "$3,931.39",
"picture": "http://placehold.it/32x32",
"age": 21,
"eyeColor": "brown",
"name": {
"first": "Rosetta",
"last": "Combs"
},
"company": "BULLJUICE",
"email": "rosetta.combs@bulljuice.co.uk",
"phone": "+1 (923) 586-3358",
"address": "460 Metropolitan Avenue, Choctaw, Connecticut, 9509",
"about": "Amet aute eu laboris laboris labore ad ullamco. Officia sint est sunt aliquip sit anim officia ad commodo eu minim. Fugiat commodo irure ex officia enim irure culpa adipisicing commodo proident sit qui. Ipsum nostrud labore proident officia aliquip Lorem. Nulla incididunt elit exercitation fugiat. Proident do occaecat voluptate Lorem esse nulla fugiat Lorem et enim minim est laboris nostrud. Minim cillum tempor non commodo veniam consectetur Lorem.",
"registered": "Saturday, November 15, 2014 9:58 AM",
"latitude": "-89.497087",
"longitude": "58.641381",
"tags": [
"ut",
"duis",
"laboris",
"amet",
"consectetur"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Mary Pratt"
},
{
"id": 1,
"name": "Heidi Contreras"
},
{
"id": 2,
"name": "Bowman Turner"
}
],
"greeting": "Hello, Rosetta! You have 6 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b039d0f556f7cd23c",
"index": 10,
"guid": "3a0babb2-c560-4f0d-b9d6-5eea8c8a589b",
"isActive": false,
"balance": "$2,076.41",
"picture": "http://placehold.it/32x32",
"age": 24,
"eyeColor": "blue",
"name": {
"first": "Mclean",
"last": "Burgess"
},
"company": "PARAGONIA",
"email": "mclean.burgess@paragonia.info",
"phone": "+1 (816) 469-3301",
"address": "476 Tompkins Place, Corinne, Massachusetts, 4786",
"about": "Anim proident consectetur magna ea ut nisi proident reprehenderit irure cupidatat eiusmod enim. Ipsum quis ex excepteur et duis in consectetur. Anim ipsum sunt officia aute culpa anim ea eiusmod et cupidatat duis qui consequat duis. Occaecat amet voluptate cupidatat amet et proident reprehenderit commodo labore aliqua ad ipsum.",
"registered": "Tuesday, January 14, 2020 1:48 PM",
"latitude": "69.255636",
"longitude": "-108.329439",
"tags": [
"nulla",
"esse",
"in",
"enim",
"duis"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Hilary Maldonado"
},
{
"id": 1,
"name": "Marks Mckenzie"
},
{
"id": 2,
"name": "Copeland Blackwell"
}
],
"greeting": "Hello, Mclean! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b70ace32b3943f724",
"index": 11,
"guid": "462245c4-3e90-4322-b7b3-14a08916da39",
"isActive": false,
"balance": "$3,126.62",
"picture": "http://placehold.it/32x32",
"age": 28,
"eyeColor": "green",
"name": {
"first": "Richmond",
"last": "Gamble"
},
"company": "ANIVET",
"email": "richmond.gamble@anivet.net",
"phone": "+1 (832) 440-2534",
"address": "291 Anthony Street, Goldfield, Kentucky, 3027",
"about": "Qui qui cupidatat est ex duis sit officia dolore cupidatat proident cillum fugiat ullamco tempor. In duis eu exercitation aliquip pariatur enim dolore voluptate id. Exercitation fugiat excepteur ut labore cillum occaecat incididunt magna proident nulla labore tempor sunt aliqua. Sint nostrud sunt dolore proident nisi aliqua anim enim sit. Velit labore minim deserunt sunt aliquip magna laboris minim nostrud aliquip aliquip elit incididunt. Labore ipsum excepteur velit pariatur quis laboris reprehenderit sint sit incididunt elit Lorem dolore laboris.",
"registered": "Wednesday, July 11, 2018 1:13 AM",
"latitude": "-37.087575",
"longitude": "-88.35237",
"tags": [
"elit",
"cillum",
"nulla",
"culpa",
"ea"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Branch French"
},
{
"id": 1,
"name": "Grimes Oneal"
},
{
"id": 2,
"name": "Edna Carney"
}
],
"greeting": "Hello, Richmond! You have 6 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b6fc2ba64ed6f1f8f",
"index": 12,
"guid": "38ae70c4-f07b-41a0-9ef0-4d8eeee92449",
"isActive": false,
"balance": "$3,981.52",
"picture": "http://placehold.it/32x32",
"age": 34,
"eyeColor": "blue",
"name": {
"first": "Faye",
"last": "Mayo"
},
"company": "PHARMEX",
"email": "faye.mayo@pharmex.com",
"phone": "+1 (871) 596-3699",
"address": "251 Russell Street, Bowden, Alaska, 2297",
"about": "Quis labore nisi voluptate do consectetur nulla deserunt ut id nulla eu velit. Eiusmod aute cillum amet fugiat quis eiusmod quis minim aute. Non culpa consequat Lorem eu enim velit et.",
"registered": "Saturday, December 17, 2016 7:29 PM",
"latitude": "-34.410867",
"longitude": "-9.991183",
"tags": [
"do",
"minim",
"deserunt",
"officia",
"aliqua"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Edwards Chan"
},
{
"id": 1,
"name": "Juliet Blackburn"
},
{
"id": 2,
"name": "Harrell Owens"
}
],
"greeting": "Hello, Faye! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b7b5606adf4e574a7",
"index": 13,
"guid": "dec7a91b-e62a-4ea0-8cf5-4aea2ac39187",
"isActive": false,
"balance": "$1,667.90",
"picture": "http://placehold.it/32x32",
"age": 34,
"eyeColor": "blue",
"name": {
"first": "Ramona",
"last": "Guerra"
},
"company": "PHOTOBIN",
"email": "ramona.guerra@photobin.biz",
"phone": "+1 (817) 549-2016",
"address": "660 Holmes Lane, Loyalhanna, Oklahoma, 619",
"about": "Irure anim labore dolore magna irure aliqua officia ipsum amet ea esse irure pariatur. Culpa cillum sunt enim cupidatat ea ullamco proident ut. Aute officia ad ad consectetur dolore esse. Minim dolor qui veniam ipsum esse eiusmod tempor laboris esse adipisicing exercitation dolor. Sit laboris ex Lorem Lorem commodo ullamco. Cillum incididunt magna ullamco cupidatat laboris proident consectetur ut reprehenderit ad mollit nulla. Deserunt incididunt esse ea dolor minim nostrud tempor.",
"registered": "Monday, April 27, 2020 3:19 AM",
"latitude": "66.607832",
"longitude": "101.158989",
"tags": [
"commodo",
"id",
"fugiat",
"sit",
"nulla"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Gonzalez Lucas"
},
{
"id": 1,
"name": "Lena Burnett"
},
{
"id": 2,
"name": "Carver Frye"
}
],
"greeting": "Hello, Ramona! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8bf420f04e7dc7649a",
"index": 14,
"guid": "47fa1f5e-8eb5-4524-b861-db5be9170c84",
"isActive": true,
"balance": "$3,941.49",
"picture": "http://placehold.it/32x32",
"age": 25,
"eyeColor": "green",
"name": {
"first": "Virginia",
"last": "Wong"
},
"company": "CRUSTATIA",
"email": "virginia.wong@crustatia.biz",
"phone": "+1 (971) 521-3909",
"address": "824 Columbia Place, Katonah, New Mexico, 2099",
"about": "Laboris sit velit nisi velit ex ex fugiat et esse nulla. Excepteur reprehenderit labore deserunt qui qui minim duis anim ipsum aute ipsum dolor veniam. Fugiat esse veniam consequat ex non ullamco consequat anim eiusmod occaecat velit eiusmod.",
"registered": "Saturday, August 8, 2020 12:13 AM",
"latitude": "-18.658717",
"longitude": "-69.908712",
"tags": [
"quis",
"dolore",
"eu",
"aute",
"fugiat"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kenya Frederick"
},
{
"id": 1,
"name": "Caroline Morse"
},
{
"id": 2,
"name": "Palmer Ochoa"
}
],
"greeting": "Hello, Virginia! You have 9 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b0f34923c4f06af6b",
"index": 15,
"guid": "4159155a-a832-4897-bccf-d629351e10a1",
"isActive": true,
"balance": "$1,075.35",
"picture": "http://placehold.it/32x32",
"age": 35,
"eyeColor": "blue",
"name": {
"first": "Jefferson",
"last": "Molina"
},
"company": "COLUMELLA",
"email": "jefferson.molina@columella.ca",
"phone": "+1 (983) 440-3501",
"address": "214 Ridgecrest Terrace, Reno, Indiana, 2243",
"about": "Ipsum veniam commodo amet tempor ipsum irure eiusmod. In sint id nulla irure dolore ea elit est esse enim sunt cupidatat. Commodo duis sit sunt deserunt mollit reprehenderit magna veniam ex sint. Et duis non aliqua consectetur cillum eiusmod esse fugiat esse.",
"registered": "Friday, July 10, 2015 7:21 PM",
"latitude": "0.301581",
"longitude": "29.741223",
"tags": [
"deserunt",
"eu",
"dolore",
"occaecat",
"est"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Judy Patrick"
},
{
"id": 1,
"name": "Boone Cameron"
},
{
"id": 2,
"name": "Lee Larson"
}
],
"greeting": "Hello, Jefferson! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b3ef97cf7f211a826",
"index": 16,
"guid": "8f6a15dc-9523-448a-b530-5351b644668c",
"isActive": true,
"balance": "$3,901.27",
"picture": "http://placehold.it/32x32",
"age": 34,
"eyeColor": "brown",
"name": {
"first": "Buck",
"last": "Fischer"
},
"company": "XELEGYL",
"email": "buck.fischer@xelegyl.tv",
"phone": "+1 (950) 426-3994",
"address": "189 Fairview Place, Lloyd, Virgin Islands, 8983",
"about": "Proident et amet cillum enim in dolor amet consectetur minim. Minim anim eu laboris ullamco ad officia voluptate quis amet cupidatat tempor. In id sint reprehenderit minim nostrud est est esse cillum anim deserunt ullamco.",
"registered": "Friday, February 21, 2020 9:19 AM",
"latitude": "42.913777",
"longitude": "115.159467",
"tags": [
"ea",
"incididunt",
"exercitation",
"voluptate",
"magna"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Lawson Schultz"
},
{
"id": 1,
"name": "Ana Knowles"
},
{
"id": 2,
"name": "Tamra Alvarado"
}
],
"greeting": "Hello, Buck! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b0342d1f73160b64a",
"index": 17,
"guid": "e1cf979b-3ccf-4e64-9c95-f0d551badec8",
"isActive": false,
"balance": "$1,315.13",
"picture": "http://placehold.it/32x32",
"age": 29,
"eyeColor": "blue",
"name": {
"first": "Klein",
"last": "Roberts"
},
"company": "TALAE",
"email": "klein.roberts@talae.name",
"phone": "+1 (839) 512-3460",
"address": "136 Coventry Road, Cobbtown, Virginia, 3111",
"about": "Incididunt quis minim sit tempor elit anim labore fugiat aliquip duis. Adipisicing laboris elit amet nostrud nulla occaecat ut. Amet culpa ea eiusmod consequat minim occaecat ad occaecat elit cupidatat qui occaecat deserunt sint. Velit adipisicing voluptate dolore pariatur culpa veniam. Magna tempor proident nulla ut minim esse. Est sint proident sunt culpa pariatur ea reprehenderit dolore laborum irure non consequat et.",
"registered": "Monday, June 10, 2019 2:18 PM",
"latitude": "13.582133",
"longitude": "174.600335",
"tags": [
"aliquip",
"enim",
"veniam",
"in",
"mollit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Tina Kaufman"
},
{
"id": 1,
"name": "Clay Morrison"
},
{
"id": 2,
"name": "Bradford Reed"
}
],
"greeting": "Hello, Klein! You have 6 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b1177392f6a39bbd2",
"index": 18,
"guid": "12be850b-a205-4d84-8a89-240b0b050b2c",
"isActive": false,
"balance": "$1,580.97",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "brown",
"name": {
"first": "Maddox",
"last": "Gilliam"
},
"company": "AQUASSEUR",
"email": "maddox.gilliam@aquasseur.me",
"phone": "+1 (950) 567-2553",
"address": "953 Henry Street, Ivanhoe, California, 5670",
"about": "In ullamco commodo consectetur incididunt aliqua sit laborum ex. Deserunt officia magna nisi aliqua commodo proident anim veniam. Incididunt magna minim anim eiusmod consequat officia cupidatat.",
"registered": "Saturday, June 3, 2017 12:45 AM",
"latitude": "-68.532028",
"longitude": "-60.388271",
"tags": [
"eiusmod",
"elit",
"culpa",
"amet",
"cupidatat"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kelli Parrish"
},
{
"id": 1,
"name": "Morrow Clements"
},
{
"id": 2,
"name": "Lynda Finley"
}
],
"greeting": "Hello, Maddox! You have 7 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b3d7426c6499c5b93",
"index": 19,
"guid": "024e021f-9cf7-45de-a6a9-9f3a1a9fbe1e",
"isActive": false,
"balance": "$3,612.95",
"picture": "http://placehold.it/32x32",
"age": 20,
"eyeColor": "blue",
"name": {
"first": "Garza",
"last": "Dillard"
},
"company": "COMCUBINE",
"email": "garza.dillard@comcubine.org",
"phone": "+1 (878) 567-2009",
"address": "955 Nassau Street, Edgar, Wyoming, 1636",
"about": "Nulla adipisicing ut amet magna in. Voluptate voluptate id amet irure anim incididunt enim. Quis cupidatat cillum mollit magna esse ipsum voluptate aute voluptate deserunt Lorem commodo do. Occaecat consequat ex magna eu aliquip magna mollit esse quis sint eu voluptate. Et do nostrud minim fugiat ex officia ipsum velit dolore. Laborum minim minim ipsum ex ipsum in qui dolor id dolore et.",
"registered": "Sunday, November 19, 2017 9:13 AM",
"latitude": "22.129737",
"longitude": "22.912609",
"tags": [
"mollit",
"eu",
"ea",
"laborum",
"sit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Callahan Myers"
},
{
"id": 1,
"name": "Stephanie Fernandez"
},
{
"id": 2,
"name": "Emilia Mccarthy"
}
],
"greeting": "Hello, Garza! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8be336614180d0b50b",
"index": 20,
"guid": "71b8d576-567d-479e-a4c4-d3d8adf0594a",
"isActive": true,
"balance": "$3,811.47",
"picture": "http://placehold.it/32x32",
"age": 39,
"eyeColor": "blue",
"name": {
"first": "Shepherd",
"last": "Bennett"
},
"company": "FLOTONIC",
"email": "shepherd.bennett@flotonic.io",
"phone": "+1 (889) 406-2225",
"address": "451 Melba Court, Rosine, Georgia, 7132",
"about": "Est commodo Lorem laboris velit in eiusmod tempor minim. Irure eiusmod ex elit nulla do et eu dolor reprehenderit. Eu sint do ad incididunt officia exercitation excepteur elit aliqua consectetur aliqua ex.",
"registered": "Monday, December 10, 2018 12:03 AM",
"latitude": "-59.828671",
"longitude": "93.11201",
"tags": [
"deserunt",
"laboris",
"sint",
"pariatur",
"pariatur"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Nicholson Oneil"
},
{
"id": 1,
"name": "Guadalupe Robbins"
},
{
"id": 2,
"name": "Erma Patterson"
}
],
"greeting": "Hello, Shepherd! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8bfbd450247d691669",
"index": 21,
"guid": "04755af6-a7e8-422c-a8f7-8fd6f9d7a0eb",
"isActive": true,
"balance": "$1,801.27",
"picture": "http://placehold.it/32x32",
"age": 23,
"eyeColor": "brown",
"name": {
"first": "Lois",
"last": "Thompson"
},
"company": "ZENTRY",
"email": "lois.thompson@zentry.co.uk",
"phone": "+1 (882) 546-2508",
"address": "563 Albany Avenue, Blandburg, Maine, 5866",
"about": "Aliquip nulla ex laboris laboris minim nisi et ipsum minim nisi Lorem qui ullamco nostrud. Et esse minim cillum labore exercitation aliquip aliquip eiusmod labore cillum commodo consectetur. Amet magna aliquip officia elit quis consequat reprehenderit eu id sint dolor. Eu eiusmod occaecat consectetur officia eiusmod excepteur excepteur voluptate adipisicing. Est nostrud voluptate irure ea. Cillum do sit proident duis officia aliquip anim et officia do.",
"registered": "Friday, November 6, 2015 6:25 AM",
"latitude": "-20.881745",
"longitude": "90.842755",
"tags": [
"dolore",
"esse",
"officia",
"ea",
"in"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Adrienne Rios"
},
{
"id": 1,
"name": "Hardin Yates"
},
{
"id": 2,
"name": "Natasha Bowen"
}
],
"greeting": "Hello, Lois! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8badb82b067152964c",
"index": 22,
"guid": "ec90dfad-a832-4f44-bd66-e7cb3065b97b",
"isActive": false,
"balance": "$2,587.20",
"picture": "http://placehold.it/32x32",
"age": 35,
"eyeColor": "green",
"name": {
"first": "Lynne",
"last": "Curtis"
},
"company": "MUSANPOLY",
"email": "lynne.curtis@musanpoly.info",
"phone": "+1 (859) 442-3725",
"address": "496 Ash Street, Grimsley, Michigan, 3706",
"about": "Culpa incididunt non et et irure ipsum est aliquip occaecat mollit. Aute nostrud fugiat ut consequat consectetur nisi ullamco do consectetur commodo minim. Occaecat ullamco amet officia laborum magna adipisicing qui consectetur culpa quis. Adipisicing pariatur nostrud ut cupidatat occaecat enim nostrud elit veniam laborum laborum veniam. Ullamco amet minim in enim proident commodo cillum ea. Eu ipsum in laboris ut. Quis officia irure in elit dolore incididunt aliquip ipsum.",
"registered": "Wednesday, January 15, 2020 3:01 PM",
"latitude": "24.640116",
"longitude": "-2.611117",
"tags": [
"ut",
"irure",
"minim",
"occaecat",
"officia"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kay Roach"
},
{
"id": 1,
"name": "Kathrine Whitney"
},
{
"id": 2,
"name": "Dunlap Montgomery"
}
],
"greeting": "Hello, Lynne! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b1f3a94d32b426459",
"index": 23,
"guid": "9e84e47b-3210-4035-b750-4862eaac6a19",
"isActive": true,
"balance": "$2,315.90",
"picture": "http://placehold.it/32x32",
"age": 38,
"eyeColor": "brown",
"name": {
"first": "Phelps",
"last": "Walker"
},
"company": "MOREGANIC",
"email": "phelps.walker@moreganic.net",
"phone": "+1 (921) 561-2473",
"address": "398 Hart Place, Clarktown, Missouri, 9834",
"about": "Nostrud amet officia tempor eiusmod nulla esse nostrud eu eiusmod ullamco nisi exercitation cupidatat. Anim irure et dolore reprehenderit cupidatat irure. Ex commodo veniam nisi sint aliquip proident elit. Consequat excepteur eu ea duis culpa occaecat ipsum elit nostrud ullamco nisi qui.",
"registered": "Sunday, November 3, 2019 7:42 AM",
"latitude": "85.204018",
"longitude": "89.891229",
"tags": [
"adipisicing",
"aute",
"anim",
"anim",
"ullamco"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Swanson Joseph"
},
{
"id": 1,
"name": "Osborne Mcbride"
},
{
"id": 2,
"name": "Cameron Brewer"
}
],
"greeting": "Hello, Phelps! You have 9 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8bc41a4e6106ec5438",
"index": 24,
"guid": "8c9ef1f8-a029-4e8a-bfd1-2a13ec6d6b42",
"isActive": false,
"balance": "$3,647.76",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "blue",
"name": {
"first": "Grant",
"last": "Burch"
},
"company": "DIGIPRINT",
"email": "grant.burch@digiprint.com",
"phone": "+1 (858) 536-3610",
"address": "510 Ralph Avenue, Kylertown, North Dakota, 1300",
"about": "Non laborum pariatur culpa eu et laborum. Ut exercitation sint fugiat reprehenderit aute excepteur id ut qui ex. Aliquip exercitation enim ullamco aute reprehenderit. Minim officia reprehenderit ullamco eu sunt pariatur irure excepteur anim. Anim veniam tempor dolore duis consequat sit nisi nisi laboris sunt veniam culpa nisi. Incididunt pariatur occaecat consequat amet enim eiusmod amet amet proident id occaecat adipisicing laboris ad.",
"registered": "Tuesday, October 27, 2015 11:57 PM",
"latitude": "60.004351",
"longitude": "-149.456916",
"tags": [
"amet",
"Lorem",
"proident",
"occaecat",
"magna"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Jacobson Maynard"
},
{
"id": 1,
"name": "Eve Berg"
},
{
"id": 2,
"name": "Leann Foley"
}
],
"greeting": "Hello, Grant! You have 5 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b12c14ad1efc4b7be",
"index": 25,
"guid": "66aef0da-6ba7-4b40-8535-e48b398d01b8",
"isActive": false,
"balance": "$1,842.10",
"picture": "http://placehold.it/32x32",
"age": 36,
"eyeColor": "green",
"name": {
"first": "Rosalind",
"last": "Hensley"
},
"company": "EZENTIA",
"email": "rosalind.hensley@ezentia.biz",
"phone": "+1 (901) 443-2585",
"address": "417 Bristol Street, Tampico, Palau, 8733",
"about": "Tempor Lorem amet ut voluptate anim eu ullamco exercitation anim ullamco cillum in aliquip esse. Voluptate nulla culpa non amet culpa in id tempor cupidatat minim sint. Aliquip consequat sunt aliqua pariatur ipsum veniam incididunt cupidatat cillum. Ea sint exercitation commodo quis aute voluptate. Irure aliqua elit tempor nisi incididunt proident consectetur tempor culpa culpa velit incididunt ea. Sunt enim cupidatat aute nulla sit ut aute cupidatat cillum ad consequat. Dolore nisi id reprehenderit ex minim enim labore laboris deserunt.",
"registered": "Sunday, October 18, 2020 10:49 PM",
"latitude": "-2.989189",
"longitude": "-113.016557",
"tags": [
"nisi",
"ut",
"minim",
"anim",
"consectetur"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Luann Horton"
},
{
"id": 1,
"name": "Liliana Jarvis"
},
{
"id": 2,
"name": "Wilda Velez"
}
],
"greeting": "Hello, Rosalind! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b8f93e606d802cb49",
"index": 26,
"guid": "0157fb91-2221-4347-846e-01f00105ea60",
"isActive": false,
"balance": "$3,949.46",
"picture": "http://placehold.it/32x32",
"age": 28,
"eyeColor": "green",
"name": {
"first": "Bernadette",
"last": "Wilcox"
},
"company": "PORTALIS",
"email": "bernadette.wilcox@portalis.biz",
"phone": "+1 (931) 435-3045",
"address": "883 Moore Street, Ripley, South Carolina, 3471",
"about": "Cillum ad quis et dolore commodo ipsum ut culpa sint eiusmod. Anim cillum excepteur reprehenderit eiusmod non id reprehenderit fugiat culpa velit deserunt et aliqua duis. Ea amet id esse enim est id eiusmod cupidatat voluptate. Veniam dolor cillum et id eiusmod mollit voluptate nisi quis aliqua aliqua minim aliqua. Quis laboris fugiat incididunt irure aliqua minim Lorem consectetur nostrud ipsum. Exercitation veniam voluptate amet aliqua ullamco nostrud commodo. Qui non exercitation adipisicing ut culpa labore.",
"registered": "Sunday, March 11, 2018 5:09 PM",
"latitude": "64.039433",
"longitude": "81.67273",
"tags": [
"voluptate",
"ipsum",
"culpa",
"sunt",
"officia"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Byrd Livingston"
},
{
"id": 1,
"name": "Carole Moss"
},
{
"id": 2,
"name": "Juarez Freeman"
}
],
"greeting": "Hello, Bernadette! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b6817642ff1ed67d4",
"index": 27,
"guid": "913c66a8-8274-48d6-8955-9bfd52d645b8",
"isActive": false,
"balance": "$3,049.96",
"picture": "http://placehold.it/32x32",
"age": 21,
"eyeColor": "blue",
"name": {
"first": "Burch",
"last": "Chen"
},
"company": "IMPERIUM",
"email": "burch.chen@imperium.ca",
"phone": "+1 (997) 537-3278",
"address": "766 Oriental Court, Brenton, Wisconsin, 4874",
"about": "Nulla dolor amet enim laboris exercitation cillum laboris exercitation eu. Culpa dolor pariatur aliqua nostrud nisi anim tempor aliquip excepteur veniam Lorem anim magna minim. Exercitation dolore nisi in laboris dolore anim officia. Lorem cillum commodo pariatur eu velit dolore veniam reprehenderit. Pariatur eu consectetur nostrud laborum. Officia irure minim et consequat dolor velit aliqua esse commodo. Voluptate ipsum in mollit officia Lorem do in cupidatat officia anim.",
"registered": "Friday, September 22, 2017 3:32 PM",
"latitude": "11.429404",
"longitude": "61.356673",
"tags": [
"commodo",
"sint",
"exercitation",
"sint",
"laboris"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Dolly Dodson"
},
{
"id": 1,
"name": "Abigail Hancock"
},
{
"id": 2,
"name": "Crystal Golden"
}
],
"greeting": "Hello, Burch! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b2f25bf473f415796",
"index": 28,
"guid": "8e16e6b2-7ab9-448d-a0f5-2c921debdb30",
"isActive": true,
"balance": "$3,241.28",
"picture": "http://placehold.it/32x32",
"age": 36,
"eyeColor": "blue",
"name": {
"first": "Aimee",
"last": "Rivers"
},
"company": "ZIZZLE",
"email": "aimee.rivers@zizzle.tv",
"phone": "+1 (989) 467-2268",
"address": "712 Bergen Place, Jacksonburg, Kansas, 3632",
"about": "Magna ad officia consectetur cupidatat sit pariatur occaecat velit eu ullamco aliquip reprehenderit sunt. In minim incididunt ullamco et enim amet velit officia culpa in exercitation eiusmod qui. Sint deserunt velit ullamco adipisicing ex. Culpa ea qui eu cupidatat reprehenderit. Tempor excepteur aliquip ad eu tempor exercitation tempor nisi pariatur enim cillum sit officia.",
"registered": "Sunday, September 17, 2017 2:41 PM",
"latitude": "-55.389515",
"longitude": "-17.00026",
"tags": [
"qui",
"id",
"excepteur",
"id",
"non"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Rogers Harrell"
},
{
"id": 1,
"name": "Marjorie Mann"
},
{
"id": 2,
"name": "Amie Parks"
}
],
"greeting": "Hello, Aimee! You have 9 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b5152cd3c67e35521",
"index": 29,
"guid": "bf1ecf35-538d-4ee7-8749-17dd8ae6a376",
"isActive": false,
"balance": "$2,749.94",
"picture": "http://placehold.it/32x32",
"age": 22,
"eyeColor": "brown",
"name": {
"first": "Shelby",
"last": "Stephenson"
},
"company": "IMANT",
"email": "shelby.stephenson@imant.name",
"phone": "+1 (971) 437-3387",
"address": "831 Terrace Place, Accoville, Guam, 373",
"about": "Cillum aliqua dolore ipsum consequat do nulla non do dolor exercitation et sunt officia. Sunt sit amet deserunt ex cupidatat sint mollit tempor exercitation aliquip est commodo. In Lorem reprehenderit consectetur consequat consectetur exercitation duis sunt. Aliqua minim voluptate non magna et deserunt fugiat. Non voluptate labore irure voluptate aliquip cillum commodo elit excepteur irure laboris Lorem deserunt sit.",
"registered": "Saturday, October 27, 2018 4:57 PM",
"latitude": "76.119072",
"longitude": "-106.920223",
"tags": [
"aliqua",
"dolor",
"non",
"do",
"commodo"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Rivas Mcleod"
},
{
"id": 1,
"name": "Lynette Holloway"
},
{
"id": 2,
"name": "Wong Giles"
}
],
"greeting": "Hello, Shelby! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8bf6c2826fcbd1f9a4",
"index": 30,
"guid": "7ae269dd-742d-48c7-bcbc-ee9d98f9a4c1",
"isActive": true,
"balance": "$3,971.40",
"picture": "http://placehold.it/32x32",
"age": 20,
"eyeColor": "brown",
"name": {
"first": "Mckinney",
"last": "Ayers"
},
"company": "ISOLOGICA",
"email": "mckinney.ayers@isologica.me",
"phone": "+1 (903) 462-2424",
"address": "746 Waldorf Court, Como, Texas, 6089",
"about": "Magna aliqua consequat nisi consectetur in irure cupidatat fugiat labore quis nostrud occaecat ea. Do deserunt eiusmod velit do anim do ex est. Magna ad minim laborum adipisicing amet dolor non nostrud. Eiusmod laboris eiusmod aliqua est magna fugiat. Sit commodo sunt anim irure occaecat eu laborum laborum cillum id cillum dolore eiusmod. Incididunt et incididunt esse consectetur consequat ut. Ipsum ut eu exercitation occaecat nisi sit irure tempor nulla do ipsum.",
"registered": "Monday, April 17, 2017 5:25 AM",
"latitude": "-76.455145",
"longitude": "71.300843",
"tags": [
"sit",
"ipsum",
"ad",
"dolor",
"adipisicing"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Cross Gaines"
},
{
"id": 1,
"name": "Erika Day"
},
{
"id": 2,
"name": "Rhodes Goodwin"
}
],
"greeting": "Hello, Mckinney! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b7c279b4eafbe5174",
"index": 31,
"guid": "eb7b77f0-48a1-439e-943f-8d7b7027cb4e",
"isActive": false,
"balance": "$3,264.45",
"picture": "http://placehold.it/32x32",
"age": 37,
"eyeColor": "brown",
"name": {
"first": "Kathie",
"last": "Horne"
},
"company": "BALOOBA",
"email": "kathie.horne@balooba.org",
"phone": "+1 (948) 469-3223",
"address": "946 Cleveland Street, Hamilton, Vermont, 8817",
"about": "Magna aliquip dolor nulla eiusmod non sit enim ipsum officia do nostrud sint irure. Mollit ea do ut Lorem consequat consequat qui nisi magna ipsum. Labore tempor nisi aliqua excepteur qui adipisicing nostrud. Elit anim officia fugiat laborum cupidatat. Elit nisi nostrud ipsum pariatur cupidatat anim proident.",
"registered": "Sunday, September 15, 2019 3:58 AM",
"latitude": "-36.030413",
"longitude": "47.370645",
"tags": [
"tempor",
"Lorem",
"cillum",
"irure",
"aliquip"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Eliza Mills"
},
{
"id": 1,
"name": "Cathryn Garrett"
},
{
"id": 2,
"name": "Ball Riddle"
}
],
"greeting": "Hello, Kathie! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8bdc73a5fa284c0f84",
"index": 32,
"guid": "83bc0071-9a94-4a6e-aba8-d9bfca99d168",
"isActive": false,
"balance": "$2,557.84",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "green",
"name": {
"first": "Kristen",
"last": "Wiggins"
},
"company": "INSURETY",
"email": "kristen.wiggins@insurety.io",
"phone": "+1 (870) 497-2411",
"address": "585 Banner Avenue, Fruitdale, Louisiana, 2906",
"about": "Enim ut occaecat aute ex laboris sunt exercitation dolor eu est et. Cupidatat tempor nisi Lorem est aliqua magna magna id id voluptate deserunt aliqua sint cillum. Voluptate consequat dolore cillum qui dolor sunt elit nulla nostrud enim enim est dolor cillum. Veniam cillum cillum tempor sit nisi ea id incididunt Lorem sint aute non consequat nostrud. Sunt incididunt aute aliquip amet voluptate commodo non laborum pariatur nostrud.",
"registered": "Thursday, September 24, 2015 6:06 AM",
"latitude": "-23.894598",
"longitude": "163.349032",
"tags": [
"dolore",
"ut",
"amet",
"esse",
"deserunt"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Jessie Ramsey"
},
{
"id": 1,
"name": "Janet Snider"
},
{
"id": 2,
"name": "Keller Castillo"
}
],
"greeting": "Hello, Kristen! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b8169f5cc6a27d301",
"index": 33,
"guid": "ffca6959-8f83-4cad-870c-07a41f5a1fb6",
"isActive": true,
"balance": "$2,323.69",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "brown",
"name": {
"first": "Ingram",
"last": "Robles"
},
"company": "IMAGEFLOW",
"email": "ingram.robles@imageflow.co.uk",
"phone": "+1 (989) 526-3200",
"address": "691 Lincoln Terrace, Fedora, Puerto Rico, 898",
"about": "Ad labore occaecat nisi ut magna id commodo deserunt labore dolore nisi elit officia. Pariatur nisi laboris tempor aliquip consequat laboris amet pariatur voluptate. Mollit anim sunt ut non. Ex in aute tempor nulla pariatur cupidatat reprehenderit sunt.",
"registered": "Tuesday, December 22, 2015 8:48 AM",
"latitude": "77.544434",
"longitude": "81.361434",
"tags": [
"consequat",
"in",
"voluptate",
"velit",
"minim"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Sharpe Bryant"
},
{
"id": 1,
"name": "Tracy Sutton"
},
{
"id": 2,
"name": "Le Pitts"
}
],
"greeting": "Hello, Ingram! You have 9 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8bedcdc6488b9281fc",
"index": 34,
"guid": "b8281046-70e3-42a6-9289-acd4d9072d28",
"isActive": true,
"balance": "$2,701.80",
"picture": "http://placehold.it/32x32",
"age": 37,
"eyeColor": "green",
"name": {
"first": "Buckley",
"last": "Monroe"
},
"company": "EXERTA",
"email": "buckley.monroe@exerta.info",
"phone": "+1 (833) 538-3206",
"address": "355 Bushwick Place, Tuttle, Arizona, 3651",
"about": "Irure consequat nostrud laborum duis nulla nostrud laboris elit. Irure proident velit id ad. Elit nisi exercitation ipsum elit enim. Ullamco reprehenderit sunt eu adipisicing occaecat magna exercitation. Duis est cupidatat ipsum culpa aute laborum in tempor aliquip. Do non occaecat irure ut dolore.",
"registered": "Saturday, February 29, 2020 10:28 AM",
"latitude": "57.28389",
"longitude": "-30.552531",
"tags": [
"est",
"officia",
"cupidatat",
"voluptate",
"minim"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Myrna Vance"
},
{
"id": 1,
"name": "Kristina Potts"
},
{
"id": 2,
"name": "Norma Bradley"
}
],
"greeting": "Hello, Buckley! You have 8 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b5385050f04da047f",
"index": 35,
"guid": "bdf5038b-2890-4936-a42b-e55ba71ae4be",
"isActive": false,
"balance": "$1,211.75",
"picture": "http://placehold.it/32x32",
"age": 20,
"eyeColor": "brown",
"name": {
"first": "Rita",
"last": "Rowe"
},
"company": "PHOLIO",
"email": "rita.rowe@pholio.net",
"phone": "+1 (873) 417-2399",
"address": "984 Sunnyside Court, Glenshaw, Hawaii, 9515",
"about": "Duis sit eu culpa dolor qui officia officia aliquip cillum do. Aliquip ullamco do enim cupidatat. Cupidatat excepteur consequat esse non consectetur proident fugiat labore aliqua. Excepteur ut in deserunt officia. Sunt ullamco Lorem nostrud incididunt voluptate pariatur nisi exercitation Lorem commodo in. Esse qui cupidatat tempor adipisicing deserunt sunt irure. Ea ut aliqua labore laboris enim aliqua cupidatat velit occaecat qui eu.",
"registered": "Sunday, February 22, 2015 7:34 PM",
"latitude": "51.52706",
"longitude": "71.392506",
"tags": [
"consectetur",
"elit",
"anim",
"ut",
"qui"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Ida Knapp"
},
{
"id": 1,
"name": "Rowena Howell"
},
{
"id": 2,
"name": "Winters Wilder"
}
],
"greeting": "Hello, Rita! You have 5 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b1ce1c6740d215ca2",
"index": 36,
"guid": "52560eec-3aa7-4c7e-911c-c34765df6bf2",
"isActive": false,
"balance": "$1,658.79",
"picture": "http://placehold.it/32x32",
"age": 37,
"eyeColor": "blue",
"name": {
"first": "Mccray",
"last": "Butler"
},
"company": "TASMANIA",
"email": "mccray.butler@tasmania.com",
"phone": "+1 (993) 419-2605",
"address": "190 Just Court, Norvelt, Florida, 7243",
"about": "Sunt eu mollit anim consectetur aliquip anim. Nostrud ex sint aute ipsum excepteur. Pariatur dolor Lorem dolore voluptate id ea sint commodo nostrud nisi.",
"registered": "Sunday, May 19, 2019 12:09 PM",
"latitude": "-41.419718",
"longitude": "170.385524",
"tags": [
"Lorem",
"esse",
"quis",
"id",
"esse"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Mccall Lowery"
},
{
"id": 1,
"name": "Gertrude Sosa"
},
{
"id": 2,
"name": "Bates Deleon"
}
],
"greeting": "Hello, Mccray! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8b9c68b956d44834c5",
"index": 37,
"guid": "81b8631a-0972-48f8-a61e-ae37a61822fe",
"isActive": false,
"balance": "$2,695.52",
"picture": "http://placehold.it/32x32",
"age": 23,
"eyeColor": "green",
"name": {
"first": "Bartlett",
"last": "Alvarez"
},
"company": "ZAJ",
"email": "bartlett.alvarez@zaj.biz",
"phone": "+1 (884) 528-3883",
"address": "416 Kay Court, Dragoon, New Hampshire, 4158",
"about": "Dolor commodo adipisicing dolore voluptate anim irure aute nisi deserunt ut et. Do labore excepteur Lorem quis incididunt esse pariatur et. Deserunt amet laboris officia proident pariatur sit aute velit nostrud elit sit exercitation laboris tempor.",
"registered": "Thursday, February 11, 2016 9:15 PM",
"latitude": "44.996231",
"longitude": "88.584748",
"tags": [
"do",
"esse",
"fugiat",
"irure",
"adipisicing"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Humphrey Campos"
},
{
"id": 1,
"name": "Albert George"
},
{
"id": 2,
"name": "Eula Figueroa"
}
],
"greeting": "Hello, Bartlett! You have 5 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081cb8b19dc339a6be63c13",
"index": 38,
"guid": "15301083-8cfe-4662-b3bd-bd18b5fd10a2",
"isActive": false,
"balance": "$2,782.01",
"picture": "http://placehold.it/32x32",
"age": 33,
"eyeColor": "blue",
"name": {
"first": "Joan",
"last": "Snyder"
},
"company": "COMTRAIL",
"email": "joan.snyder@comtrail.biz",
"phone": "+1 (959) 547-2261",
"address": "687 Decatur Street, Sardis, Oregon, 1139",
"about": "Officia aliquip culpa tempor enim culpa Lorem aliquip amet et. Nisi incididunt incididunt ex consequat. Aliqua veniam in ex commodo minim ea Lorem occaecat ipsum culpa sit cupidatat Lorem enim. Et et dolore sit excepteur aliqua sint tempor nulla nisi et est esse.",
"registered": "Sunday, June 3, 2018 10:00 PM",
"latitude": "-81.797609",
"longitude": "-163.590231",
"tags": [
"dolor",
"proident",
"ex",
"do",
"excepteur"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Leah Malone"
},
{
"id": 1,
"name": "Stafford Mueller"
},
{
"id": 2,
"name": "Moses Flynn"
}
],
"greeting": "Hello, Joan! You have 6 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081cb8bf3fb291a492e9d6f",
"index": 39,
"guid": "b438c327-0754-4cd9-893f-44bb4f70ff8f",
"isActive": false,
"balance": "$3,339.83",
"picture": "http://placehold.it/32x32",
"age": 35,
"eyeColor": "green",
"name": {
"first": "Walls",
"last": "Duncan"
},
"company": "LUNCHPOD",
"email": "walls.duncan@lunchpod.ca",
"phone": "+1 (909) 480-2815",
"address": "400 Adler Place, Johnsonburg, American Samoa, 4999",
"about": "Ipsum et excepteur incididunt culpa quis sunt. Amet aliqua minim amet reprehenderit enim tempor laboris culpa pariatur. Anim non ut culpa voluptate laborum elit amet amet est consectetur occaecat. Tempor non deserunt voluptate pariatur ad voluptate qui deserunt.",
"registered": "Friday, July 28, 2017 3:40 PM",
"latitude": "43.383283",
"longitude": "-14.755056",
"tags": [
"amet",
"incididunt",
"laborum",
"adipisicing",
"ipsum"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Justice Whitley"
},
{
"id": 1,
"name": "Rosa Elliott"
},
{
"id": 2,
"name": "Savannah Hubbard"
}
],
"greeting": "Hello, Walls! You have 8 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081cb8b5548547134401f25",
"index": 40,
"guid": "ee3d9765-bbde-4992-b38b-d5c238b2df52",
"isActive": true,
"balance": "$3,159.25",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "green",
"name": {
"first": "Terri",
"last": "Spencer"
},
"company": "MALATHION",
"email": "terri.spencer@malathion.tv",
"phone": "+1 (894) 538-2362",
"address": "522 Wolcott Street, Idledale, West Virginia, 4639",
"about": "Deserunt adipisicing incididunt occaecat elit. Adipisicing cillum est magna quis sint quis aliqua amet in minim dolor incididunt mollit. Exercitation enim aute velit laboris ex minim esse et laborum cillum tempor sunt. Incididunt esse eu aliquip labore fugiat eiusmod elit ullamco velit aute ullamco. Et minim nisi voluptate ut nostrud labore. Nostrud culpa consectetur fugiat minim eu in enim ipsum eiusmod et laboris. Tempor officia tempor tempor proident dolor voluptate exercitation commodo velit quis fugiat sint duis et.",
"registered": "Monday, November 28, 2016 2:22 PM",
"latitude": "37.542052",
"longitude": "164.708052",
"tags": [
"est",
"ex",
"adipisicing",
"officia",
"non"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Gabrielle Mejia"
},
{
"id": 1,
"name": "Ray Bartlett"
},
{
"id": 2,
"name": "Susana Trujillo"
}
],
"greeting": "Hello, Terri! You have 10 unread messages.",
"favoriteFruit": "strawberry"
}
]

View File

@ -0,0 +1,1655 @@
[
{
"_id": "6081ce6acf61f83263bb83a1",
"index": 0,
"guid": "5777e5c0-586a-4e97-affd-495e3e82085f",
"isActive": true,
"balance": "$3,308.69",
"picture": "http://placehold.it/32x32",
"age": 30,
"eyeColor": "green",
"name": {
"first": "Bond",
"last": "Carney"
},
"company": "DATACATOR",
"email": "bond.carney@datacator.io",
"phone": "+1 (980) 405-2647",
"address": "828 Commercial Street, Snyderville, Tennessee, 9840",
"about": "按时间先后进行排序 Ipsum eiusmod ad pariatur aute amet dolore eiusmod fugiat. Ex deserunt pariatur aliqua consectetur id non ullamco ut consectetur consectetur nostrud est amet. Occaecat nisi officia anim duis veniam anim dolore incididunt proident ea in ullamco. Eu quis excepteur et sint cupidatat eiusmod esse. Ea in ex tempor irure est. Anim et aute anim aute cillum mollit duis ullamco officia irure nostrud nulla. Nulla enim magna eiusmod eiusmod irure nulla esse dolore pariatur non laborum.",
"registered": "Thursday, November 12, 2015 10:10 AM",
"latitude": "12.52655",
"longitude": "17.161964",
"tags": [
"nulla",
"ex",
"Lorem",
"fugiat",
"velit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Smith Ayers"
},
{
"id": 1,
"name": "Valentine Whitley"
},
{
"id": 2,
"name": "Leona Stout"
}
],
"greeting": "Hello, Bond! You have 7 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6aa0248fd3377c4dcf",
"index": 1,
"guid": "ffe82f9a-0240-4966-99ae-85a70e631411",
"isActive": false,
"balance": "$1,055.74",
"picture": "http://placehold.it/32x32",
"age": 28,
"eyeColor": "brown",
"name": {
"first": "Glenna",
"last": "Kaufman"
},
"company": "GEEKWAGON",
"email": "glenna.kaufman@geekwagon.me",
"phone": "+1 (951) 583-2667",
"address": "848 Cleveland Street, Brantleyville, Alaska, 8255",
"about": "Ut ullamco veniam dolore do eu enim dolor incididunt magna labore esse laboris. Labore in ipsum amet cupidatat sit occaecat velit mollit eiusmod id fugiat occaecat. Labore deserunt ad incididunt sit ea velit magna officia ex.",
"registered": "Wednesday, December 16, 2015 11:18 AM",
"latitude": "64.346624",
"longitude": "-26.840163",
"tags": [
"dolor",
"elit",
"magna",
"dolore",
"nisi"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Powell Mccullough"
},
{
"id": 1,
"name": "Nelda Chen"
},
{
"id": 2,
"name": "Rachelle Dillard"
}
],
"greeting": "Hello, Glenna! You have 7 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6ae3a83136741ad238",
"index": 2,
"guid": "49cdc238-5d64-44b1-a1b2-5b22915ba8c7",
"isActive": false,
"balance": "$1,233.64",
"picture": "http://placehold.it/32x32",
"age": 36,
"eyeColor": "green",
"name": {
"first": "Margo",
"last": "Wolf"
},
"company": "OMATOM",
"email": "margo.wolf@omatom.org",
"phone": "+1 (882) 581-2606",
"address": "635 McKibben Street, Templeton, Florida, 412",
"about": "Consequat ipsum ut voluptate ullamco velit cillum do ullamco. Incididunt anim sunt amet occaecat. Incididunt magna eu et id culpa do sit velit nulla ea deserunt. Dolor veniam ipsum officia sint velit tempor quis consectetur dolor exercitation duis ea. Ullamco in adipisicing officia nisi. Incididunt cupidatat magna voluptate occaecat cupidatat commodo aliqua.",
"registered": "Monday, January 4, 2016 1:49 AM",
"latitude": "15.039771",
"longitude": "-90.075062",
"tags": [
"anim",
"sit",
"officia",
"incididunt",
"duis"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kinney Carroll"
},
{
"id": 1,
"name": "Beth Sawyer"
},
{
"id": 2,
"name": "Latasha Valentine"
}
],
"greeting": "Hello, Margo! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a2f63014fcc038a16",
"index": 3,
"guid": "09d096f7-ea67-4eee-b072-436b5a237c59",
"isActive": false,
"balance": "$2,708.88",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "green",
"name": {
"first": "Hancock",
"last": "Sykes"
},
"company": "COMVERGES",
"email": "hancock.sykes@comverges.info",
"phone": "+1 (925) 448-2400",
"address": "184 Locust Avenue, Coaldale, New Hampshire, 2936",
"about": "Aliquip veniam amet consequat ex in occaecat. Consequat tempor cupidatat sunt aute Lorem aliquip occaecat do nisi adipisicing et. Ex duis ad et eiusmod consequat sit excepteur ut.",
"registered": "Thursday, December 18, 2014 1:28 PM",
"latitude": "-73.389766",
"longitude": "149.77773",
"tags": [
"reprehenderit",
"nostrud",
"sunt",
"duis",
"aute"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kim Le"
},
{
"id": 1,
"name": "Debra Hampton"
},
{
"id": 2,
"name": "Blanca Burks"
}
],
"greeting": "Hello, Hancock! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a009368407e589218",
"index": 4,
"guid": "0c99c353-d266-4877-b3af-676380242b62",
"isActive": false,
"balance": "$1,605.80",
"picture": "http://placehold.it/32x32",
"age": 28,
"eyeColor": "blue",
"name": {
"first": "Norris",
"last": "Brady"
},
"company": "TERRAGEN",
"email": "norris.brady@terragen.ca",
"phone": "+1 (843) 541-3555",
"address": "591 Putnam Avenue, Escondida, California, 4965",
"about": "Voluptate tempor ullamco exercitation reprehenderit in veniam. Veniam amet quis amet tempor do ipsum officia dolore eu exercitation sit consequat aliquip veniam. Reprehenderit non excepteur reprehenderit sint officia. Ut dolore cupidatat duis sint eu laborum in enim officia eu. Sunt exercitation pariatur sint officia eu dolor minim ut quis nisi.",
"registered": "Saturday, March 3, 2018 5:35 AM",
"latitude": "-70.618492",
"longitude": "-91.542494",
"tags": [
"ipsum",
"qui",
"sunt",
"dolor",
"cupidatat"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Huff Terry"
},
{
"id": 1,
"name": "Bowman Padilla"
},
{
"id": 2,
"name": "Jerry Collier"
}
],
"greeting": "Hello, Norris! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a3f97c1c5c2d444a9",
"index": 5,
"guid": "96e6071e-c321-4068-825b-b9005138fc8d",
"isActive": true,
"balance": "$2,777.15",
"picture": "http://placehold.it/32x32",
"age": 35,
"eyeColor": "brown",
"name": {
"first": "Luz",
"last": "Dotson"
},
"company": "POSHOME",
"email": "luz.dotson@poshome.name",
"phone": "+1 (905) 564-3120",
"address": "288 Maujer Street, Temperanceville, Wyoming, 9214",
"about": "Nostrud tempor exercitation tempor Lorem aliqua Lorem aute. Sunt nulla dolore ullamco eiusmod cillum in eu. Ex reprehenderit velit qui id qui incididunt dolore est Lorem non. Tempor occaecat eiusmod ipsum ex Lorem laboris tempor non ad ad sunt sint. Laboris est occaecat dolor do velit enim laborum. Aute ad excepteur excepteur eu. Id aliqua cupidatat aliqua proident voluptate eu labore aliqua ex duis incididunt proident esse ut.",
"registered": "Sunday, January 3, 2021 11:13 AM",
"latitude": "-34.196073",
"longitude": "75.503513",
"tags": [
"ullamco",
"enim",
"consectetur",
"magna",
"voluptate"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Bartlett Hartman"
},
{
"id": 1,
"name": "Candace Sherman"
},
{
"id": 2,
"name": "Burch Rice"
}
],
"greeting": "Hello, Luz! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6a8098b10fdcdcd320",
"index": 6,
"guid": "9b919a1f-ad9d-4c4d-ad12-9b1a9208b152",
"isActive": false,
"balance": "$1,928.24",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "green",
"name": {
"first": "Bridges",
"last": "Mckee"
},
"company": "MAKINGWAY",
"email": "bridges.mckee@makingway.tv",
"phone": "+1 (976) 506-3536",
"address": "424 Apollo Street, Smeltertown, Hawaii, 8334",
"about": "Adipisicing veniam officia minim duis aute duis aliqua culpa fugiat consequat dolore do. Eiusmod cupidatat tempor in ea mollit ullamco consectetur. Nisi quis qui sit proident aute ullamco id pariatur. Velit eiusmod qui eu cillum proident anim qui. Est ad dolor mollit ea proident eiusmod eiusmod ullamco proident. Culpa adipisicing cupidatat et sit aliquip incididunt veniam aliquip adipisicing ut aliquip aute eu est. Tempor ullamco labore quis exercitation occaecat in elit excepteur.",
"registered": "Thursday, April 27, 2017 10:02 PM",
"latitude": "54.274178",
"longitude": "2.449225",
"tags": [
"est",
"id",
"sint",
"occaecat",
"eu"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Gaines Newton"
},
{
"id": 1,
"name": "William Mendoza"
},
{
"id": 2,
"name": "Liz Stanley"
}
],
"greeting": "Hello, Bridges! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6ac6e8953e602a2ac2",
"index": 7,
"guid": "27845f3f-8fda-4040-a84d-a3ce423657fb",
"isActive": true,
"balance": "$2,980.64",
"picture": "http://placehold.it/32x32",
"age": 24,
"eyeColor": "green",
"name": {
"first": "Short",
"last": "Hebert"
},
"company": "ANIMALIA",
"email": "short.hebert@animalia.net",
"phone": "+1 (949) 546-3885",
"address": "980 Guider Avenue, Roland, Virgin Islands, 5496",
"about": "Non velit laboris incididunt duis adipisicing voluptate est Lorem laboris magna minim adipisicing. Officia culpa pariatur sint deserunt excepteur cillum ipsum commodo excepteur adipisicing. Commodo mollit proident aliqua ullamco tempor do aute. Enim duis fugiat fugiat ex excepteur sunt deserunt nisi occaecat consectetur. Amet exercitation ex cupidatat aute exercitation culpa ad occaecat occaecat qui duis est.",
"registered": "Saturday, March 12, 2016 9:14 AM",
"latitude": "-75.343149",
"longitude": "43.853993",
"tags": [
"proident",
"culpa",
"laboris",
"est",
"dolor"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Hazel Maynard"
},
{
"id": 1,
"name": "Ava Williams"
},
{
"id": 2,
"name": "Emily Harrison"
}
],
"greeting": "Hello, Short! You have 6 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a3b361dd7bd64e6c8",
"index": 8,
"guid": "fbaf20d2-f1df-4314-a63b-35d2c8912f01",
"isActive": true,
"balance": "$1,053.16",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "green",
"name": {
"first": "Elsa",
"last": "Boyle"
},
"company": "PLUTORQUE",
"email": "elsa.boyle@plutorque.co.uk",
"phone": "+1 (801) 553-3869",
"address": "957 Cheever Place, Allentown, Alabama, 6069",
"about": "Consectetur exercitation ad ex pariatur voluptate deserunt. Veniam eu do enim ea nisi irure cillum reprehenderit do sint. Id do amet consequat pariatur sunt mollit ipsum magna elit reprehenderit id. Sit culpa aliquip mollit anim deserunt. In proident eu aliqua consectetur voluptate aliqua enim duis. Aliquip reprehenderit consectetur incididunt cillum voluptate consequat officia amet do aute dolore ut ea id.",
"registered": "Sunday, October 12, 2014 6:07 PM",
"latitude": "57.837987",
"longitude": "-80.20426",
"tags": [
"et",
"occaecat",
"quis",
"nostrud",
"exercitation"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Mercer Lara"
},
{
"id": 1,
"name": "Tamika Melendez"
},
{
"id": 2,
"name": "Ofelia Moreno"
}
],
"greeting": "Hello, Elsa! You have 7 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6ae4b88b8e5d9abf23",
"index": 9,
"guid": "a65a8904-4f05-4aef-b8e3-a8207e44cec6",
"isActive": true,
"balance": "$3,640.90",
"picture": "http://placehold.it/32x32",
"age": 24,
"eyeColor": "brown",
"name": {
"first": "Joan",
"last": "Townsend"
},
"company": "UXMOX",
"email": "joan.townsend@uxmox.us",
"phone": "+1 (835) 579-2679",
"address": "800 Metrotech Courtr, Greensburg, Vermont, 3593",
"about": "Irure incididunt deserunt officia fugiat quis ex cupidatat dolor sint veniam. Occaecat fugiat exercitation ad deserunt ad veniam in cillum magna consectetur Lorem proident. Do veniam nulla veniam ad tempor dolor minim dolore. Do aute nostrud quis tempor consequat pariatur reprehenderit exercitation amet nulla. Fugiat ex deserunt proident voluptate proident excepteur do sit. Anim dolore fugiat voluptate do deserunt ut nulla amet.",
"registered": "Tuesday, May 5, 2020 3:26 AM",
"latitude": "38.214051",
"longitude": "-5.297229",
"tags": [
"cupidatat",
"eu",
"excepteur",
"do",
"ea"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Hutchinson Berg"
},
{
"id": 1,
"name": "Gloria Kramer"
},
{
"id": 2,
"name": "Shelia Benjamin"
}
],
"greeting": "Hello, Joan! You have 9 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6a48f0c3c826d9a566",
"index": 10,
"guid": "936ee7cb-c007-4d33-805d-57a23646a7b1",
"isActive": false,
"balance": "$1,235.66",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "brown",
"name": {
"first": "Nina",
"last": "Lyons"
},
"company": "FUTURITY",
"email": "nina.lyons@futurity.com",
"phone": "+1 (973) 585-2174",
"address": "871 Woods Place, Wanship, New Jersey, 5451",
"about": "Aliqua in dolore deserunt proident anim officia quis occaecat cillum dolore. Ad culpa irure est laborum non nisi aliquip excepteur. Cupidatat consequat ea do tempor consequat nostrud pariatur est duis ad voluptate.",
"registered": "Saturday, February 10, 2018 7:41 AM",
"latitude": "57.974419",
"longitude": "153.080488",
"tags": [
"cupidatat",
"adipisicing",
"duis",
"culpa",
"magna"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Walter Olsen"
},
{
"id": 1,
"name": "Autumn Wiley"
},
{
"id": 2,
"name": "Kris Dodson"
}
],
"greeting": "Hello, Nina! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a17b0643baafd1e8d",
"index": 11,
"guid": "15593e42-f82d-4883-8a6e-dcebb44333d4",
"isActive": true,
"balance": "$1,204.72",
"picture": "http://placehold.it/32x32",
"age": 27,
"eyeColor": "brown",
"name": {
"first": "Jaclyn",
"last": "Wade"
},
"company": "RODEOLOGY",
"email": "jaclyn.wade@rodeology.biz",
"phone": "+1 (836) 426-2145",
"address": "519 Campus Road, Zortman, Puerto Rico, 7770",
"about": "Fugiat ex sit do officia nostrud commodo. Minim qui commodo ullamco magna non occaecat cupidatat culpa. Quis nulla eiusmod voluptate est.",
"registered": "Wednesday, September 16, 2020 4:09 PM",
"latitude": "84.341431",
"longitude": "-25.127724",
"tags": [
"culpa",
"aliquip",
"incididunt",
"adipisicing",
"eiusmod"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Trisha Short"
},
{
"id": 1,
"name": "Downs Frost"
},
{
"id": 2,
"name": "Marisol Sweet"
}
],
"greeting": "Hello, Jaclyn! You have 9 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6ac7b99bfb7539207f",
"index": 12,
"guid": "3d5367ed-1fdf-4954-a54f-294a06bbf62c",
"isActive": false,
"balance": "$3,878.65",
"picture": "http://placehold.it/32x32",
"age": 29,
"eyeColor": "blue",
"name": {
"first": "Marshall",
"last": "Wood"
},
"company": "NSPIRE",
"email": "marshall.wood@nspire.io",
"phone": "+1 (964) 534-3029",
"address": "607 Clara Street, Lavalette, Mississippi, 6880",
"about": "Ad nulla ad excepteur tempor enim velit incididunt non Lorem est dolor labore voluptate velit. Laboris eiusmod enim cupidatat veniam irure et deserunt consequat excepteur irure eiusmod in sit commodo. Nisi veniam voluptate proident nostrud qui dolore exercitation aliquip. Reprehenderit anim deserunt culpa duis. Reprehenderit nisi ullamco ullamco sunt velit velit et cillum aute laborum esse culpa.",
"registered": "Wednesday, July 29, 2020 4:20 PM",
"latitude": "-25.715105",
"longitude": "-100.452323",
"tags": [
"magna",
"ipsum",
"aliquip",
"duis",
"minim"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Teresa Bates"
},
{
"id": 1,
"name": "Martin Sargent"
},
{
"id": 2,
"name": "Hubbard Ratliff"
}
],
"greeting": "Hello, Marshall! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a8ebb5c3ce22aac77",
"index": 13,
"guid": "05589ec6-c058-428d-8b67-26e814ef08bf",
"isActive": false,
"balance": "$2,133.86",
"picture": "http://placehold.it/32x32",
"age": 24,
"eyeColor": "blue",
"name": {
"first": "Beulah",
"last": "Bishop"
},
"company": "SOPRANO",
"email": "beulah.bishop@soprano.me",
"phone": "+1 (864) 465-3913",
"address": "569 Eckford Street, Gasquet, New York, 1657",
"about": "Deserunt reprehenderit fugiat dolor adipisicing excepteur commodo do sit culpa duis non ad. Proident laborum et culpa sunt ullamco fugiat. Excepteur occaecat occaecat ipsum nostrud nostrud. Occaecat dolor consequat reprehenderit proident incididunt anim labore deserunt. Aliquip proident sint commodo commodo quis in. Anim commodo dolor amet eiusmod commodo non ea excepteur nulla ut. Ullamco ex sint amet do tempor ipsum.",
"registered": "Sunday, April 17, 2016 9:24 PM",
"latitude": "51.93969",
"longitude": "-79.157282",
"tags": [
"excepteur",
"anim",
"excepteur",
"mollit",
"in"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Parks Middleton"
},
{
"id": 1,
"name": "Schroeder Velazquez"
},
{
"id": 2,
"name": "Candice Summers"
}
],
"greeting": "Hello, Beulah! You have 10 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6a2f779af9d598d6a3",
"index": 14,
"guid": "789dd521-f5e6-4764-8a5d-bd176f93bc91",
"isActive": true,
"balance": "$3,627.81",
"picture": "http://placehold.it/32x32",
"age": 32,
"eyeColor": "blue",
"name": {
"first": "Marva",
"last": "Holt"
},
"company": "KIDGREASE",
"email": "marva.holt@kidgrease.org",
"phone": "+1 (946) 572-2890",
"address": "274 Rewe Street, Dragoon, North Dakota, 8396",
"about": "Voluptate labore sit proident sunt ut magna irure consequat enim sit occaecat. Cillum sint voluptate occaecat minim cillum. Sunt nulla ut ex aliqua duis minim.",
"registered": "Wednesday, March 1, 2017 5:42 PM",
"latitude": "-22.669367",
"longitude": "160.405114",
"tags": [
"tempor",
"nulla",
"officia",
"amet",
"do"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Rebecca Rodgers"
},
{
"id": 1,
"name": "Dawn Aguirre"
},
{
"id": 2,
"name": "Watson Boone"
}
],
"greeting": "Hello, Marva! You have 9 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6a884afa3321d44e02",
"index": 15,
"guid": "3d1bc167-5bae-49f9-97ca-b4f5f36fecba",
"isActive": false,
"balance": "$1,132.05",
"picture": "http://placehold.it/32x32",
"age": 32,
"eyeColor": "brown",
"name": {
"first": "Carol",
"last": "Bell"
},
"company": "SCHOOLIO",
"email": "carol.bell@schoolio.info",
"phone": "+1 (918) 480-3419",
"address": "896 Osborn Street, Grapeview, Marshall Islands, 9524",
"about": "Adipisicing voluptate eiusmod sint eu nisi enim anim. Nulla ullamco ex ut qui labore proident est ad. Occaecat tempor ut magna ipsum magna non nostrud ea.",
"registered": "Monday, January 12, 2015 2:32 AM",
"latitude": "-17.873203",
"longitude": "95.273932",
"tags": [
"dolore",
"incididunt",
"Lorem",
"excepteur",
"veniam"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Greene Gray"
},
{
"id": 1,
"name": "Kelley Powell"
},
{
"id": 2,
"name": "Morrison Ortega"
}
],
"greeting": "Hello, Carol! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a51b6913d64c6bc50",
"index": 16,
"guid": "919f6475-66d5-4af3-961c-71a4e2d34dc7",
"isActive": true,
"balance": "$1,640.29",
"picture": "http://placehold.it/32x32",
"age": 21,
"eyeColor": "brown",
"name": {
"first": "Gretchen",
"last": "Wynn"
},
"company": "NIKUDA",
"email": "gretchen.wynn@nikuda.ca",
"phone": "+1 (832) 423-2234",
"address": "661 Veterans Avenue, Idledale, Texas, 6744",
"about": "Fugiat voluptate velit cupidatat et in non amet. Eu non qui pariatur incididunt eu id ex. Dolor ea ea ea magna anim. Ipsum anim aliquip elit duis pariatur veniam qui dolor dolor magna est eu in. Cupidatat deserunt ullamco ipsum deserunt dolor deserunt ipsum labore adipisicing. Reprehenderit dolore ex dolor tempor ut. Eiusmod aliqua fugiat minim commodo nostrud commodo consectetur esse nisi irure.",
"registered": "Tuesday, September 13, 2016 2:19 AM",
"latitude": "69.441831",
"longitude": "-106.947933",
"tags": [
"dolore",
"mollit",
"veniam",
"veniam",
"enim"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Carney Cabrera"
},
{
"id": 1,
"name": "Dorothea Bullock"
},
{
"id": 2,
"name": "Hunt Finch"
}
],
"greeting": "Hello, Gretchen! You have 9 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a0e92563e59c800a4",
"index": 17,
"guid": "cec535e9-6578-43f0-b7dc-602871acb2f6",
"isActive": false,
"balance": "$3,128.21",
"picture": "http://placehold.it/32x32",
"age": 25,
"eyeColor": "brown",
"name": {
"first": "Morin",
"last": "Hinton"
},
"company": "AQUASURE",
"email": "morin.hinton@aquasure.name",
"phone": "+1 (926) 484-3552",
"address": "349 Fayette Street, Disautel, Northern Mariana Islands, 1456",
"about": "Incididunt ex sint qui enim consectetur magna consequat dolore magna culpa aliquip est. Ullamco aute ea do excepteur. Id excepteur minim adipisicing laborum et voluptate Lorem id adipisicing tempor. Ad culpa elit quis irure nostrud. Voluptate duis est id ullamco adipisicing culpa laboris non fugiat exercitation labore. Enim incididunt labore cupidatat velit qui laboris in dolore cillum dolor excepteur.",
"registered": "Friday, June 26, 2015 7:08 AM",
"latitude": "-60.033067",
"longitude": "-53.713602",
"tags": [
"ullamco",
"aliqua",
"ad",
"velit",
"occaecat"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Munoz Nicholson"
},
{
"id": 1,
"name": "Manuela Howell"
},
{
"id": 2,
"name": "Noelle Rodriguez"
}
],
"greeting": "Hello, Morin! You have 5 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6aac76ba77db5be019",
"index": 18,
"guid": "8bca30de-e609-423a-a84e-881b75795235",
"isActive": true,
"balance": "$1,511.59",
"picture": "http://placehold.it/32x32",
"age": 38,
"eyeColor": "blue",
"name": {
"first": "Flynn",
"last": "Rodriquez"
},
"company": "SLOGANAUT",
"email": "flynn.rodriquez@sloganaut.tv",
"phone": "+1 (812) 537-3546",
"address": "746 Cypress Avenue, Blanco, Virginia, 3306",
"about": "Tempor quis reprehenderit culpa consectetur. Ipsum deserunt cillum labore aliqua quis ut. Ipsum deserunt Lorem tempor Lorem ullamco cillum minim aliqua. Ut minim nostrud nulla sint magna Lorem eu deserunt.",
"registered": "Monday, April 27, 2020 8:20 PM",
"latitude": "36.872841",
"longitude": "5.866665",
"tags": [
"incididunt",
"proident",
"velit",
"laborum",
"aliquip"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Anderson Fuentes"
},
{
"id": 1,
"name": "Beck Berry"
},
{
"id": 2,
"name": "Lester Kim"
}
],
"greeting": "Hello, Flynn! You have 8 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a56ec741be03743a9",
"index": 19,
"guid": "cf68b879-4562-4453-8c83-7440963189ea",
"isActive": true,
"balance": "$2,182.90",
"picture": "http://placehold.it/32x32",
"age": 24,
"eyeColor": "green",
"name": {
"first": "Paige",
"last": "Hickman"
},
"company": "INFOTRIPS",
"email": "paige.hickman@infotrips.net",
"phone": "+1 (978) 412-2744",
"address": "831 Forrest Street, Yorklyn, Iowa, 4572",
"about": "Enim proident voluptate labore duis culpa eiusmod mollit amet. Irure ullamco in Lorem aliquip ea magna labore in eiusmod ea veniam sint. Sit in veniam proident aute incididunt. Reprehenderit minim sunt ex voluptate elit aute ad. Commodo amet in laborum enim ipsum officia exercitation commodo. Velit enim sunt elit exercitation ex duis eu pariatur minim.",
"registered": "Monday, February 10, 2020 6:13 AM",
"latitude": "-35.855946",
"longitude": "0.310791",
"tags": [
"minim",
"ut",
"reprehenderit",
"pariatur",
"tempor"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Ina Keller"
},
{
"id": 1,
"name": "Coleen Jarvis"
},
{
"id": 2,
"name": "Clarice Anderson"
}
],
"greeting": "Hello, Paige! You have 5 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6a5348645699d2658c",
"index": 20,
"guid": "58805bf9-82ec-4778-94ca-7cc882db05e7",
"isActive": true,
"balance": "$1,033.73",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "green",
"name": {
"first": "Moses",
"last": "Levine"
},
"company": "PORTALIS",
"email": "moses.levine@portalis.co.uk",
"phone": "+1 (990) 519-3510",
"address": "849 Polar Street, Ribera, Michigan, 7503",
"about": "Ea commodo nisi non laborum enim velit ex reprehenderit officia non cupidatat. Ex ex minim adipisicing nulla dolor est ullamco est reprehenderit. Eiusmod voluptate laborum cillum cupidatat nostrud ex ullamco occaecat occaecat. Est nostrud aliquip exercitation ut. Non qui sint aute ut excepteur. Sit nulla duis Lorem ex proident consectetur eiusmod sint consequat nostrud officia sunt. Sint labore dolor in ipsum quis aute sint anim do.",
"registered": "Sunday, November 23, 2014 7:06 AM",
"latitude": "17.016121",
"longitude": "-55.207637",
"tags": [
"voluptate",
"officia",
"deserunt",
"in",
"mollit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Kenya Carpenter"
},
{
"id": 1,
"name": "Richard Guthrie"
},
{
"id": 2,
"name": "Harrison Santos"
}
],
"greeting": "Hello, Moses! You have 9 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a2f220789b8f0a459",
"index": 21,
"guid": "9e6df0cc-e81c-4306-b201-eec096071564",
"isActive": false,
"balance": "$1,698.90",
"picture": "http://placehold.it/32x32",
"age": 20,
"eyeColor": "blue",
"name": {
"first": "Daniels",
"last": "Madden"
},
"company": "FROLIX",
"email": "daniels.madden@frolix.us",
"phone": "+1 (809) 477-2199",
"address": "971 Seaview Court, Chical, South Dakota, 2307",
"about": "Aliquip exercitation eu quis commodo exercitation nulla quis anim ipsum aute. Nisi irure elit dolore sint laboris quis incididunt dolore elit officia eiusmod laborum. Id magna labore excepteur exercitation duis anim.",
"registered": "Sunday, June 18, 2017 8:51 AM",
"latitude": "17.980818",
"longitude": "146.291017",
"tags": [
"sint",
"eiusmod",
"veniam",
"nisi",
"eiusmod"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Keri Weiss"
},
{
"id": 1,
"name": "Navarro Nolan"
},
{
"id": 2,
"name": "Esperanza Alston"
}
],
"greeting": "Hello, Daniels! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6aa2c272cc54822f21",
"index": 22,
"guid": "12085f23-3189-4658-adaf-aaf570107965",
"isActive": true,
"balance": "$2,223.46",
"picture": "http://placehold.it/32x32",
"age": 30,
"eyeColor": "brown",
"name": {
"first": "Donaldson",
"last": "Sears"
},
"company": "EVEREST",
"email": "donaldson.sears@everest.com",
"phone": "+1 (837) 595-2050",
"address": "234 Laurel Avenue, Brambleton, Idaho, 2184",
"about": "Anim do aliqua incididunt est tempor magna eu amet tempor elit ad. Cupidatat aliquip exercitation nostrud enim do velit enim amet est exercitation id do consequat. Pariatur anim voluptate aute aliqua. Commodo Lorem sunt fugiat velit id fugiat anim duis velit minim sint Lorem. Tempor sit sit deserunt id amet deserunt. Velit magna sint est labore pariatur esse pariatur ad aliqua ipsum in. Ea officia nulla pariatur in amet in.",
"registered": "Sunday, November 29, 2020 2:11 AM",
"latitude": "-63.138526",
"longitude": "151.409341",
"tags": [
"veniam",
"quis",
"excepteur",
"est",
"elit"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Hobbs Drake"
},
{
"id": 1,
"name": "Newton Woods"
},
{
"id": 2,
"name": "Kristen Shaw"
}
],
"greeting": "Hello, Donaldson! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6af128fa3f22f57aea",
"index": 23,
"guid": "1e25fe6e-a9b9-4bc1-888d-e14e47f9a4be",
"isActive": true,
"balance": "$3,594.99",
"picture": "http://placehold.it/32x32",
"age": 33,
"eyeColor": "blue",
"name": {
"first": "Reynolds",
"last": "Baldwin"
},
"company": "CODAX",
"email": "reynolds.baldwin@codax.biz",
"phone": "+1 (981) 466-3234",
"address": "128 Cherry Street, Kula, Palau, 2987",
"about": "Sit esse velit esse culpa labore consequat voluptate anim cupidatat labore Lorem. Proident aliqua fugiat elit deserunt do incididunt. Adipisicing ut ut nisi ullamco et Lorem irure. Sint est ex sit deserunt proident laboris et quis irure ex. Anim id nisi fugiat laboris qui.",
"registered": "Tuesday, July 30, 2019 6:27 AM",
"latitude": "-44.543732",
"longitude": "168.787532",
"tags": [
"pariatur",
"minim",
"culpa",
"incididunt",
"excepteur"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Twila Mclean"
},
{
"id": 1,
"name": "Vicki Slater"
},
{
"id": 2,
"name": "Park Potter"
}
],
"greeting": "Hello, Reynolds! You have 7 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a6527c3341bce9aa0",
"index": 24,
"guid": "b8e88ab2-38c9-425b-9a67-b8f0aab7738f",
"isActive": true,
"balance": "$3,065.93",
"picture": "http://placehold.it/32x32",
"age": 28,
"eyeColor": "green",
"name": {
"first": "Burton",
"last": "Lane"
},
"company": "MALATHION",
"email": "burton.lane@malathion.io",
"phone": "+1 (976) 579-2140",
"address": "756 Sandford Street, Vernon, Louisiana, 2798",
"about": "Quis mollit ea consequat deserunt cupidatat qui sit tempor ipsum cillum amet culpa deserunt consequat. Aliquip deserunt ea do consequat aute qui do elit minim id sint magna nulla. Minim velit ut veniam incididunt esse ipsum.",
"registered": "Tuesday, March 25, 2014 11:29 PM",
"latitude": "14.53458",
"longitude": "71.625027",
"tags": [
"amet",
"fugiat",
"duis",
"mollit",
"Lorem"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Penelope Turner"
},
{
"id": 1,
"name": "Adrienne Nielsen"
},
{
"id": 2,
"name": "Shawna Daniel"
}
],
"greeting": "Hello, Burton! You have 10 unread messages.",
"favoriteFruit": "banana"
},
{
"_id": "6081ce6a84d0d39417c83d32",
"index": 25,
"guid": "ece6e71d-d318-4e9b-8e68-a5c0a88f1216",
"isActive": true,
"balance": "$2,489.87",
"picture": "http://placehold.it/32x32",
"age": 31,
"eyeColor": "brown",
"name": {
"first": "Diann",
"last": "Snider"
},
"company": "APEX",
"email": "diann.snider@apex.me",
"phone": "+1 (877) 509-3935",
"address": "788 Court Square, Navarre, Guam, 2764",
"about": "Qui cillum consectetur amet dolor consequat. Incididunt exercitation ipsum et anim non deserunt sit fugiat et amet. Elit eu laborum duis minim dolor velit irure irure officia. Mollit exercitation dolore adipisicing culpa consectetur aliquip ex proident culpa aliquip magna dolore ullamco. Occaecat do duis magna id consectetur. Et adipisicing aliqua sint qui sint eiusmod magna consectetur pariatur quis magna id id enim. Pariatur proident consectetur cillum exercitation dolor cupidatat pariatur.",
"registered": "Saturday, October 31, 2015 4:54 PM",
"latitude": "-51.144368",
"longitude": "3.183612",
"tags": [
"ipsum",
"cupidatat",
"reprehenderit",
"aute",
"eiusmod"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Dennis Glenn"
},
{
"id": 1,
"name": "Julie Murphy"
},
{
"id": 2,
"name": "Adela Love"
}
],
"greeting": "Hello, Diann! You have 9 unread messages.",
"favoriteFruit": "apple"
},
{
"_id": "6081ce6ae0ce4e80c6a2a13c",
"index": 26,
"guid": "c94fd661-018f-4885-9236-eed76a5c0bef",
"isActive": false,
"balance": "$1,921.97",
"picture": "http://placehold.it/32x32",
"age": 26,
"eyeColor": "brown",
"name": {
"first": "Wilkinson",
"last": "Miranda"
},
"company": "COMDOM",
"email": "wilkinson.miranda@comdom.org",
"phone": "+1 (963) 506-2021",
"address": "352 Cumberland Walk, Nash, Montana, 2381",
"about": "Dolore consectetur in deserunt veniam sit tempor irure. Lorem nisi incididunt deserunt ea do laborum fugiat deserunt pariatur do non. Culpa exercitation elit sunt sit mollit mollit esse non enim ad veniam. Sit culpa nisi laboris officia. Minim ut cillum aliquip in. Pariatur in nostrud deserunt elit sunt aliquip tempor enim. Exercitation minim irure occaecat mollit nisi aute consectetur irure sunt pariatur reprehenderit proident excepteur aute.",
"registered": "Thursday, September 5, 2019 12:55 PM",
"latitude": "-8.923973",
"longitude": "103.864233",
"tags": [
"tempor",
"sunt",
"ad",
"culpa",
"in"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Schultz Harmon"
},
{
"id": 1,
"name": "Jennie Lawson"
},
{
"id": 2,
"name": "Sonia Pitts"
}
],
"greeting": "Hello, Wilkinson! You have 6 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a968c369422ad6680",
"index": 27,
"guid": "b53d6221-893f-4ebd-bad2-5342b21283cf",
"isActive": false,
"balance": "$3,882.62",
"picture": "http://placehold.it/32x32",
"age": 40,
"eyeColor": "brown",
"name": {
"first": "Teri",
"last": "Vance"
},
"company": "CYTRAK",
"email": "teri.vance@cytrak.info",
"phone": "+1 (936) 523-2573",
"address": "404 Porter Avenue, Washington, Missouri, 8789",
"about": "Duis laboris consectetur proident veniam quis nostrud incididunt. Aute id qui ad fugiat est. Ullamco aute ex qui laboris commodo nostrud ea occaecat sunt exercitation aliquip aute proident. Non occaecat ad exercitation cillum esse nisi excepteur laborum esse enim ex et adipisicing officia.",
"registered": "Thursday, July 2, 2020 8:48 PM",
"latitude": "-67.491848",
"longitude": "-36.49362",
"tags": [
"fugiat",
"excepteur",
"nisi",
"incididunt",
"tempor"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Lacy Miller"
},
{
"id": 1,
"name": "Tabatha Patterson"
},
{
"id": 2,
"name": "Doris Medina"
}
],
"greeting": "Hello, Teri! You have 10 unread messages.",
"favoriteFruit": "strawberry"
},
{
"_id": "6081ce6a35e6ac4ab2785a69",
"index": 28,
"guid": "e19dba91-56fb-4c7d-8f14-7d0db8b7aa13",
"isActive": true,
"balance": "$3,771.21",
"picture": "http://placehold.it/32x32",
"age": 25,
"eyeColor": "green",
"name": {
"first": "Rodgers",
"last": "Paul"
},
"company": "EXOTERIC",
"email": "rodgers.paul@exoteric.ca",
"phone": "+1 (809) 545-3089",
"address": "714 Bartlett Street, Beechmont, West Virginia, 8267",
"about": "Irure veniam consectetur consectetur et magna amet do voluptate sint incididunt qui. Tempor non eu commodo do do commodo ea laboris occaecat velit consequat aute labore. Culpa eiusmod ullamco elit non velit ea ea. Anim consequat excepteur adipisicing occaecat id mollit elit ullamco qui amet. Consectetur consectetur eu laborum laboris et in mollit sint laboris. Non dolor sit dolore velit et ut adipisicing ex velit laboris incididunt elit minim eu.",
"registered": "Saturday, June 4, 2016 12:25 AM",
"latitude": "-24.446516",
"longitude": "-170.047284",
"tags": [
"aute",
"quis",
"consequat",
"qui",
"ipsum"
],
"range": [
0,
1,
2,
3,
4,
5,
6,
7,
8,
9
],
"friends": [
{
"id": 0,
"name": "Maricela Goodman"
},
{
"id": 1,
"name": "Hanson Nichols"
},
{
"id": 2,
"name": "Hudson Obrien"
}
],
"greeting": "Hello, Rodgers! You have 7 unread messages.",
"favoriteFruit": "banana"
}
]

View File

@ -15,4 +15,10 @@ log4j.logger.io.debezium.core=DEBUG
log4j.logger.io.debezium.connector.oracle=DEBUG
# Avoid the fallback property spam
log4j.logger.io.debezium.config.Configuration=ERROR
log4j.logger.io.debezium.config.Configuration=ERROR
# Avoid writing converter configurations
log4j.logger.org.apache.kafka.connect.json.JsonConverterConfig=WARN
log4j.logger.io.confluent.connect.avro.AvroConverterConfig=WARN
log4j.logger.io.confluent.kafka.serializers.KafkaAvroSerializerConfig=WARN
log4j.logger.io.confluent.kafka.serializers.KafkaAvroDeserializerConfig=WARN