From d730b7d3b3a9e5bb7eca4094d5ab36155a468719 Mon Sep 17 00:00:00 2001 From: Lars M Johansson Date: Mon, 29 Jul 2024 10:55:36 +0200 Subject: [PATCH] DBZ-8114 Implementing VariableScaleDecimal and improving tests --- .github/workflows/cross-maven.yml | 4 +- .github/workflows/maven.yml | 4 +- README.md | 2 +- TODO.md | 49 +- pom.xml | 7 + .../informix/InformixConnection.java | 14 +- .../informix/InformixConnectorConfig.java | 8 +- .../informix/InformixValueConverters.java | 100 +- .../docker/informix-cdc-docker/12/Dockerfile | 1 + .../informix-cdc-docker/12/informix_init.sh | 10 +- .../docker/informix-cdc-docker/12/testdb.sql | 2 + .../14/informix_post_init.sh | 10 +- .../AbstractInformixDatatypesTest.java | 674 +++++ .../informix/BlockingSnapshotIT.java | 65 +- .../informix/CustomTestSnapshot.java | 8 +- .../connector/informix/FieldTest.java | 19 + .../informix/IncrementalSnapshotIT.java | 71 +- .../informix/InformixCdcTypesIT.java | 198 -- .../informix/InformixConnectionIT.java | 3 +- .../connector/informix/NotificationsIT.java | 14 +- .../informix/OutboxEventRouterIT.java | 185 ++ .../informix/SnapshotDatatypesIT.java | 94 + .../informix/StreamingDatatypesIT.java | 63 + .../informix/TransactionMetadataIT.java | 6 +- .../connector/informix/util/TestHelper.java | 10 + src/test/resources/data/test_lob_data.json | 2339 +++++++++++++++++ 26 files changed, 3617 insertions(+), 343 deletions(-) create mode 100644 src/test/docker/informix-cdc-docker/12/testdb.sql create mode 100644 src/test/java/io/debezium/connector/informix/AbstractInformixDatatypesTest.java create mode 100644 src/test/java/io/debezium/connector/informix/FieldTest.java delete mode 100644 src/test/java/io/debezium/connector/informix/InformixCdcTypesIT.java create mode 100644 src/test/java/io/debezium/connector/informix/OutboxEventRouterIT.java create mode 100644 src/test/java/io/debezium/connector/informix/SnapshotDatatypesIT.java create mode 100644 src/test/java/io/debezium/connector/informix/StreamingDatatypesIT.java create mode 100644 src/test/resources/data/test_lob_data.json diff --git a/.github/workflows/cross-maven.yml b/.github/workflows/cross-maven.yml index bce733b..c4a4f29 100644 --- a/.github/workflows/cross-maven.yml +++ b/.github/workflows/cross-maven.yml @@ -126,6 +126,6 @@ jobs: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.httpconnectionManager.ttlSeconds=120 - -Ddebezium.test.records.waittime=7 - -Ddebezium.test.records.waittime.after.nulls=13 + -Ddebezium.test.records.waittime=10 + -Ddebezium.test.records.waittime.after.nulls=10 -DfailFlakyTests=false diff --git a/.github/workflows/maven.yml b/.github/workflows/maven.yml index edc91d7..f77b3a8 100644 --- a/.github/workflows/maven.yml +++ b/.github/workflows/maven.yml @@ -97,6 +97,6 @@ jobs: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.httpconnectionManager.ttlSeconds=120 - -Ddebezium.test.records.waittime=7 - -Ddebezium.test.records.waittime.after.nulls=13 + -Ddebezium.test.records.waittime=10 + -Ddebezium.test.records.waittime.after.nulls=10 -DfailFlakyTests=false diff --git a/README.md b/README.md index 214b4ca..d8bee22 100644 --- a/README.md +++ b/README.md @@ -1,7 +1,7 @@ [![License](http://img.shields.io/:license-apache%202.0-brightgreen.svg)](http://www.apache.org/licenses/LICENSE-2.0.html) [![Maven Central](https://maven-badges.herokuapp.com/maven-central/io.debezium/debezium-connector-informix/badge.svg)](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22io.debezium%22) -[![Build Status](https://github.com/debezium/debezium-connector-informix/actions/workflows/maven.yml/badge.svg?branch=main)](https://github.com/debezium/debezium-connector-informix/actions) +[![Build Status](https://github.com/debezium/debezium-connector-informix/actions/workflows/Maven%20CI/badge.svg?branch=main)](https://github.com/debezium/debezium-connector-informix/actions) [![User chat](https://img.shields.io/badge/chat-users-brightgreen.svg)](https://gitter.im/debezium/user) [![Developer chat](https://img.shields.io/badge/chat-devs-brightgreen.svg)](https://gitter.im/debezium/dev) [![Google Group](https://img.shields.io/:mailing%20list-debezium-brightgreen.svg)](https://groups.google.com/forum/#!forum/debezium) diff --git a/TODO.md b/TODO.md index 233ccfa..d3d8b92 100644 --- a/TODO.md +++ b/TODO.md @@ -1,47 +1,12 @@ -## 1. Fix reload of schema after restart - -## 2. Fix blocking snapshot - -## 3. Successfully determine current maximum LSN - -## 4. Testcases - -Adapt more standard testcases from Debezium parent +## ~~1. Fix reload of schema after restart~~ +## ~~2. Fix blocking snapshot~~ +## ~~3. Successfully determine current maximum LSN~~ +## ~~4. Testcases~~ +~~Adapt more standard testcases from Debezium parent~~ ## 5. Implements Metrics - Reference: https://github.com/debezium/debezium/blob/main/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlChangeEventSourceMetricsFactory.java -## 6. Decimal handling mode - -If we set "decimal.handling.mode=percision", which is the default option, it will cause the following exception: - -```text -[2022-05-15 00:48:42,552] ERROR WorkerSourceTask{id=informix-connector-214414-0} Task threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask:179) -org.apache.kafka.connect.errors.ConnectException: Tolerance exceeded in error handler - at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndHandleError(RetryWithToleranceOperator.java:178) - at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execute(RetryWithToleranceOperator.java:104) - at org.apache.kafka.connect.runtime.WorkerSourceTask.convertTransformedRecord(WorkerSourceTask.java:290) - at org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:316) - at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:240) - at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:177) - at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:227) - at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539) - at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) - at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) - at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) - at java.base/java.lang.Thread.run(Thread.java:833) -Caused by: org.apache.kafka.connect.errors.DataException: BigDecimal has mismatching scale value for given Decimal schema - at org.apache.kafka.connect.data.Decimal.fromLogical(Decimal.java:68) - at org.apache.kafka.connect.json.JsonConverter$13.toJson(JsonConverter.java:206) - at org.apache.kafka.connect.json.JsonConverter.convertToJson(JsonConverter.java:606) - at org.apache.kafka.connect.json.JsonConverter.convertToJson(JsonConverter.java:693) - at org.apache.kafka.connect.json.JsonConverter.convertToJson(JsonConverter.java:693) - at org.apache.kafka.connect.json.JsonConverter.convertToJsonWithEnvelope(JsonConverter.java:581) - at org.apache.kafka.connect.json.JsonConverter.fromConnectData(JsonConverter.java:335) - at org.apache.kafka.connect.storage.Converter.fromConnectData(Converter.java:62) - at org.apache.kafka.connect.runtime.WorkerSourceTask.lambda$convertTransformedRecord$2(WorkerSourceTask.java:290) - at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndRetry(RetryWithToleranceOperator.java:128) - at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndHandleError(RetryWithToleranceOperator.java:162) - ... 11 more -``` +## ~~6. Decimal handling mode~~ +~~If we set "decimal.handling.mode=percision", which is the default option, it will cause the following exception: ...~~ diff --git a/pom.xml b/pom.xml index 513457e..6881562 100644 --- a/pom.xml +++ b/pom.xml @@ -500,6 +500,12 @@ ${project.basedir}/src/test/docker/informix-cdc-docker/12/informix_init.sh scripts unix + 755 + + + ${project.basedir}/src/test/docker/informix-cdc-docker/12/testdb.sql + informix/etc + unix @@ -550,6 +556,7 @@ ${project.basedir}/src/test/docker/informix-cdc-docker/14/informix_post_init.sh config unix + 755 ${project.basedir}/src/test/docker/informix-cdc-docker/14/testdb.sql diff --git a/src/main/java/io/debezium/connector/informix/InformixConnection.java b/src/main/java/io/debezium/connector/informix/InformixConnection.java index 3dc6c95..99bacd2 100644 --- a/src/main/java/io/debezium/connector/informix/InformixConnection.java +++ b/src/main/java/io/debezium/connector/informix/InformixConnection.java @@ -57,7 +57,7 @@ public class InformixConnection extends JdbcConnection { private static final ConnectionFactory FACTORY = JdbcConnection.patternBasedFactory( URL_PATTERN, - IfxDriver.class.getName(), + IfxDriver.class.getCanonicalName(), InformixConnection.class.getClassLoader(), JdbcConfiguration.PORT.withDefault(InformixConnectorConfig.PORT.defaultValueAsString())); @@ -81,7 +81,7 @@ private String retrieveRealDatabaseName() { return queryAndMap(GET_DATABASE_NAME, singleResultMapper(rs -> rs.getString(1), "Could not retrieve database name")); } catch (SQLException e) { - throw new RuntimeException("Couldn't obtain database name", e); + throw new DebeziumException("Couldn't obtain database name", e); } } @@ -186,6 +186,8 @@ public String quotedColumnIdString(String columnName) { public DataSource datasource() { return new DataSource() { + private PrintWriter logWriter; + @Override public Connection getConnection() throws SQLException { return connection(); @@ -199,12 +201,12 @@ public Connection getConnection(String username, String password) throws SQLExce @Override public PrintWriter getLogWriter() { - throw new UnsupportedOperationException("getLogWriter"); + return this.logWriter; } @Override public void setLogWriter(PrintWriter out) { - throw new UnsupportedOperationException("setLogWriter"); + this.logWriter = out; } @Override @@ -214,12 +216,12 @@ public void setLoginTimeout(int seconds) { @Override public int getLoginTimeout() { - return 0; + return (int) config().getConnectionTimeout().toSeconds(); } @Override public java.util.logging.Logger getParentLogger() throws SQLFeatureNotSupportedException { - throw new SQLFeatureNotSupportedException("getParentLogger"); + return java.util.logging.Logger.getLogger("io.debezium.connector.informix"); } @Override diff --git a/src/main/java/io/debezium/connector/informix/InformixConnectorConfig.java b/src/main/java/io/debezium/connector/informix/InformixConnectorConfig.java index 6d065e3..0cd33fb 100644 --- a/src/main/java/io/debezium/connector/informix/InformixConnectorConfig.java +++ b/src/main/java/io/debezium/connector/informix/InformixConnectorConfig.java @@ -68,6 +68,7 @@ public enum SnapshotMode implements EnumeratedValue { * Perform a snapshot of the schema but no data upon initial startup of a connector. * @deprecated to be removed in Debezium 3.0, replaced by {{@link #NO_DATA}} */ + @Deprecated SCHEMA_ONLY("schema_only"), /** @@ -340,7 +341,7 @@ public String getValue() { public static final Field CDC_TIMEOUT = Field.create("cdc.timeout") .withDisplayName("CDC Engine timeout") .withType(ConfigDef.Type.INT) - .withGroup(Field.createGroupEntry(Field.Group.CONNECTOR_ADVANCED, 0)) + .withGroup(Field.createGroupEntry(Field.Group.CONNECTOR_ADVANCED, 1)) .withWidth(Width.MEDIUM) .withImportance(Importance.MEDIUM) .withDescription("Specifies a timeout to interrupt blocking to wait on an event, in seconds. " @@ -359,8 +360,8 @@ public String getValue() { PORT, USER, PASSWORD, - QUERY_TIMEOUT_MS, - DATABASE_NAME) + DATABASE_NAME, + QUERY_TIMEOUT_MS) .connector( SNAPSHOT_MODE, SNAPSHOT_ISOLATION_MODE, @@ -373,7 +374,6 @@ public String getValue() { SCHEMA_EXCLUDE_LIST, INCLUDE_SCHEMA_COMMENTS, INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, - SNAPSHOT_MAX_THREADS, DatabaseHeartbeatImpl.HEARTBEAT_ACTION_QUERY) .create(); diff --git a/src/main/java/io/debezium/connector/informix/InformixValueConverters.java b/src/main/java/io/debezium/connector/informix/InformixValueConverters.java index 2f8bc9e..b60476d 100644 --- a/src/main/java/io/debezium/connector/informix/InformixValueConverters.java +++ b/src/main/java/io/debezium/connector/informix/InformixValueConverters.java @@ -5,12 +5,22 @@ */ package io.debezium.connector.informix; +import java.io.BufferedReader; +import java.math.BigDecimal; +import java.sql.Clob; +import java.sql.SQLException; +import java.sql.Types; import java.time.ZoneOffset; +import java.util.stream.Collectors; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.SchemaBuilder; +import com.informix.jdbc.IfxCblob; + import io.debezium.config.CommonConnectorConfig.BinaryHandlingMode; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.data.VariableScaleDecimal; import io.debezium.jdbc.JdbcValueConverters; import io.debezium.jdbc.TemporalPrecisionMode; import io.debezium.relational.Column; @@ -24,6 +34,8 @@ */ public class InformixValueConverters extends JdbcValueConverters { + private static final int FLOATING_POINT_DECIMAL_SCALE = 255; + /** * Create a new instance that always uses UTC for the default time zone when * converting values without timezone information to values that require @@ -41,20 +53,106 @@ public InformixValueConverters(DecimalMode decimalMode, TemporalPrecisionMode te @Override public SchemaBuilder schemaBuilder(Column column) { + logger.debug("Building schema for column {} of type {} named {} with constraints ({},{})", + column.name(), + column.jdbcType(), + column.typeName(), + column.length(), + column.scale()); + switch (column.jdbcType()) { + case Types.NUMERIC: + case Types.DECIMAL: + return getNumericSchema(column); default: - return super.schemaBuilder(column); + SchemaBuilder builder = super.schemaBuilder(column); + logger.debug("JdbcValueConverters returned '{}' for column '{}'", builder != null ? builder.getClass().getName() : null, column.name()); + return builder; } } + private SchemaBuilder getNumericSchema(Column column) { + if (column.scale().isPresent()) { + + if (column.scale().get() == FLOATING_POINT_DECIMAL_SCALE && decimalMode == DecimalMode.PRECISE) { + return VariableScaleDecimal.builder(); + } + + return super.schemaBuilder(column); + } + + if (decimalMode == DecimalMode.PRECISE) { + return VariableScaleDecimal.builder(); + } + + if (column.length() == 0) { + // Defined as DECIMAL without specifying a length and scale, treat as DECIMAL(16) + return SpecialValueDecimal.builder(decimalMode, 16, -1); + } + + return SpecialValueDecimal.builder(decimalMode, column.length(), -1); + } + @Override public ValueConverter converter(Column column, Field fieldDefn) { switch (column.jdbcType()) { + case Types.NUMERIC: + case Types.DECIMAL: + return getNumericConverter(column, fieldDefn); default: return super.converter(column, fieldDefn); } } + private ValueConverter getNumericConverter(Column column, Field fieldDefn) { + if (column.scale().isPresent()) { + + if (column.scale().get() == FLOATING_POINT_DECIMAL_SCALE && decimalMode == DecimalMode.PRECISE) { + return data -> convertVariableScale(column, fieldDefn, data); + } + + return data -> convertNumeric(column, fieldDefn, data); + } + + return data -> convertVariableScale(column, fieldDefn, data); + } + + private Object convertVariableScale(Column column, Field fieldDefn, Object data) { + data = convertNumeric(column, fieldDefn, data); // provides default value + + if (data == null) { + return null; + } + if (decimalMode == DecimalMode.PRECISE) { + if (data instanceof SpecialValueDecimal) { + return VariableScaleDecimal.fromLogical(fieldDefn.schema(), (SpecialValueDecimal) data); + } + else if (data instanceof BigDecimal) { + return VariableScaleDecimal.fromLogical(fieldDefn.schema(), (BigDecimal) data); + } + } + else { + return data; + } + return handleUnknownData(column, fieldDefn, data); + } + + @Override + protected Object convertString(Column column, Field fieldDefn, Object data) { + if (data instanceof Clob) { + return convertValue(column, fieldDefn, data, "", (receiver) -> { + try { + receiver.deliver(new BufferedReader(((IfxCblob) data).getCharacterStream()).lines().collect(Collectors.joining(System.lineSeparator()))); + } + catch (SQLException e) { + throw new RuntimeException("Error processing data from " + column.jdbcType() + " and column " + column + + ": class=" + data.getClass(), e); + } + }); + } + return super.convertString(column, fieldDefn, data); + } + @Override protected int getTimePrecision(Column column) { return column.length() < 20 ? 0 : column.length() - 20; diff --git a/src/test/docker/informix-cdc-docker/12/Dockerfile b/src/test/docker/informix-cdc-docker/12/Dockerfile index b7f91b6..f5c0b01 100644 --- a/src/test/docker/informix-cdc-docker/12/Dockerfile +++ b/src/test/docker/informix-cdc-docker/12/Dockerfile @@ -1,6 +1,7 @@ FROM icr.io/informix/informix-developer-database:12.10.FC12W1DE ADD --chown=informix:informix informix_init.sh /opt/ibm/scripts +ADD --chown=informix:informix testdb.sql /opt/ibm/informix/etc RUN sed -Ei 's/^(USEOSTIME)\s+\S/\1 1/' /opt/ibm/informix/etc/onconfig.std diff --git a/src/test/docker/informix-cdc-docker/12/informix_init.sh b/src/test/docker/informix-cdc-docker/12/informix_init.sh index 86de071..cf99331 100644 --- a/src/test/docker/informix-cdc-docker/12/informix_init.sh +++ b/src/test/docker/informix-cdc-docker/12/informix_init.sh @@ -25,12 +25,10 @@ if [ ${iter} -gt 120 ]; then exit fi -dbaccess sysadmin $INFORMIX_DATA_DIR/extend_root.sql >>$INIT_LOG 2>&1 +dbaccess sysadmin $BASEDIR/sql/informix_extend_root.sql >>$INIT_LOG 2>&1 -if [ $DB_SBSPACE ]; then - dbaccess sysadmin $INFORMIX_DATA_DIR/sbspace.sql >>$INIT_LOG 2>&1 -fi +dbaccess sysadmin $BASEDIR/sql/informix_sbspace.sql >>$INIT_LOG 2>&1 -dbaccess < $INFORMIXDIR/etc/syscdcv1.sql +dbaccess sysadmin $INFORMIXDIR/etc/syscdcv1.sql >>$INIT_LOG 2>&1 -echo 'create database testdb with log;' | dbaccess - - +dbaccess sysadmin $INFORMIXDIR/etc/testdb.sql >>$INIT_LOG 2>&1 diff --git a/src/test/docker/informix-cdc-docker/12/testdb.sql b/src/test/docker/informix-cdc-docker/12/testdb.sql new file mode 100644 index 0000000..335c735 --- /dev/null +++ b/src/test/docker/informix-cdc-docker/12/testdb.sql @@ -0,0 +1,2 @@ +set lock mode to wait; +create database testdb with log; diff --git a/src/test/docker/informix-cdc-docker/14/informix_post_init.sh b/src/test/docker/informix-cdc-docker/14/informix_post_init.sh index e8ba5b7..c85470e 100644 --- a/src/test/docker/informix-cdc-docker/14/informix_post_init.sh +++ b/src/test/docker/informix-cdc-docker/14/informix_post_init.sh @@ -1,11 +1,9 @@ #!/bin/bash -dbaccess < $INFORMIXDIR/etc/syscdcv1.sql +dbaccess sysadmin $BASEDIR/sql/informix_extend_root.sql >>$INIT_LOG 2>&1 -dbaccess < $INFORMIXDIR/etc/testdb.sql +dbaccess sysadmin $BASEDIR/sql/informix_sbspace.sql >>$INIT_LOG 2>&1 -# cat /dev/null > $INFORMIX_DATA_DIR/spaces/rootdbs.001 > $INFORMIX_DATA_DIR/spaces/rootdbs.002 -# chmod 660 $INFORMIX_DATA_DIR/spaces/rootdbs.001 $INFORMIX_DATA_DIR/spaces/rootdbs.002 -# onspaces -c -d rootdbs1 -p $INFORMIX_DATA_DIR/spaces/rootdbs.001 -o 0 -s 350000 -# onspaces -c -d rootdbs2 -p $INFORMIX_DATA_DIR/spaces/rootdbs.002 -o 0 -s 350000 +dbaccess sysadmin $INFORMIXDIR/etc/syscdcv1.sql >>$INIT_LOG 2>&1 +dbaccess sysadmin $INFORMIXDIR/etc/testdb.sql >>$INIT_LOG 2>&1 diff --git a/src/test/java/io/debezium/connector/informix/AbstractInformixDatatypesTest.java b/src/test/java/io/debezium/connector/informix/AbstractInformixDatatypesTest.java new file mode 100644 index 0000000..c82ac89 --- /dev/null +++ b/src/test/java/io/debezium/connector/informix/AbstractInformixDatatypesTest.java @@ -0,0 +1,674 @@ +/* + * 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.informix; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.sql.Clob; +import java.sql.SQLException; +import java.time.*; +import java.util.Arrays; +import java.util.List; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import io.debezium.config.Configuration; +import io.debezium.config.Configuration.Builder; +import io.debezium.connector.informix.util.TestHelper; +import io.debezium.data.SchemaAndValueField; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.data.VariableScaleDecimal; +import io.debezium.data.VerifyRecord; +import io.debezium.doc.FixFor; +import io.debezium.embedded.async.AbstractAsyncEngineConnectorTest; +import io.debezium.jdbc.JdbcValueConverters.DecimalMode; +import io.debezium.jdbc.TemporalPrecisionMode; +import io.debezium.time.*; +import io.debezium.util.Testing; + +/** + * Integration test to verify different Oracle datatypes. + * + * @author Jiri Pechanec, Lars M Johansson + */ +public abstract class AbstractInformixDatatypesTest extends AbstractAsyncEngineConnectorTest { + + private static final String DDL_STRING = "create table type_string (" + + " id serial not null primary key, " + + " val_varchar varchar(255), " + + " val_nvarchar nvarchar(255), " + + " val_lvarchar lvarchar(1000), " + + " val_char char(3), " + + " val_nchar nchar(3)" + + ");"; + + private static final String DDL_FP = "create table type_fp (" + + " id serial not null primary key, " + + " val_sf smallfloat, " + + " val_f float, " + + " val_f_10 float (10), " + + " val_r real, " + + " val_dp double precision, " + + " val_numeric numeric(10, 6), " + + " val_decimal decimal(10, 6), " + + " val_decimal_vs decimal(10), " + + " val_decimal_vs2 decimal" + + ");"; + + private static final String DDL_INT = "create table type_int (" + + " id serial not null primary key, " + + " val_int int, " + + " val_int8 int8, " + + " val_integer integer, " + + " val_smallint smallint, " + + " val_bigint bigint, " + + " val_decimal decimal(10,0), " + + " val_numeric numeric(10,0)" + + ");"; + + private static final String DDL_TIME = "create table type_time (" + + " id serial not null primary key, " + + " val_date date, " + + " val_time datetime hour to second, " + + " val_datetime datetime year to second, " + + " val_timestamp datetime year to fraction, " + + " val_timestamp_us datetime year to fraction(5) " + + // " val_int_ytm interval year to month, " + + // " val_int_dts interval day(3) to second " + + ");"; + + private static final String DDL_CLOB = "create table type_clob (" + + " id serial not null primary key, " + + " val_clob_inline clob, " + + " val_clob_short clob, " + + " val_clob_long clob" + + ");"; + + private static final List EXPECTED_STRING = Arrays.asList( + new SchemaAndValueField("val_varchar", Schema.OPTIONAL_STRING_SCHEMA, "vc"), + new SchemaAndValueField("val_nvarchar", Schema.OPTIONAL_STRING_SCHEMA, "nvc"), + new SchemaAndValueField("val_lvarchar", Schema.OPTIONAL_STRING_SCHEMA, "lvc"), + new SchemaAndValueField("val_char", Schema.OPTIONAL_STRING_SCHEMA, "c "), + new SchemaAndValueField("val_nchar", Schema.OPTIONAL_STRING_SCHEMA, "nc ")); + + private static final List EXPECTED_FP = Arrays.asList( + new SchemaAndValueField("val_sf", Schema.OPTIONAL_FLOAT32_SCHEMA, 1.1f), + new SchemaAndValueField("val_f", Schema.OPTIONAL_FLOAT64_SCHEMA, 2.22d), + new SchemaAndValueField("val_f_10", Schema.OPTIONAL_FLOAT64_SCHEMA, 3.333d), + new SchemaAndValueField("val_r", Schema.OPTIONAL_FLOAT32_SCHEMA, 4.4444f), + new SchemaAndValueField("val_dp", Schema.OPTIONAL_FLOAT64_SCHEMA, 5.55555d), + new SchemaAndValueField("val_numeric", SpecialValueDecimal.builder(DecimalMode.PRECISE, 10, 6).optional().build(), new BigDecimal("1234.567891")), + new SchemaAndValueField("val_decimal", SpecialValueDecimal.builder(DecimalMode.PRECISE, 10, 6).optional().build(), new BigDecimal("1234.567891")), + new SchemaAndValueField("val_decimal_vs", VariableScaleDecimal.builder().optional().build(), + VariableScaleDecimal.fromLogical(VariableScaleDecimal.builder().optional().build(), new BigDecimal("77.323"))), + new SchemaAndValueField("val_decimal_vs2", VariableScaleDecimal.builder().optional().build(), + VariableScaleDecimal.fromLogical(VariableScaleDecimal.builder().optional().build(), new BigDecimal("77.323")))); + + private static final List EXPECTED_FP_AS_STRING = Arrays.asList( + new SchemaAndValueField("val_sf", Schema.OPTIONAL_FLOAT32_SCHEMA, 1.1f), + new SchemaAndValueField("val_f", Schema.OPTIONAL_FLOAT64_SCHEMA, 2.22), + new SchemaAndValueField("val_f_10", Schema.OPTIONAL_FLOAT64_SCHEMA, 3.333), + new SchemaAndValueField("val_r", Schema.OPTIONAL_FLOAT32_SCHEMA, 4.4444f), + new SchemaAndValueField("val_dp", Schema.OPTIONAL_FLOAT64_SCHEMA, 5.55555), + new SchemaAndValueField("val_numeric", Schema.OPTIONAL_STRING_SCHEMA, "1234.567891"), + new SchemaAndValueField("val_decimal", Schema.OPTIONAL_STRING_SCHEMA, "1234.567891"), + new SchemaAndValueField("val_decimal_vs", Schema.OPTIONAL_STRING_SCHEMA, "77.323"), + new SchemaAndValueField("val_decimal_vs2", Schema.OPTIONAL_STRING_SCHEMA, "77.323")); + + private static final List EXPECTED_FP_AS_DOUBLE = Arrays.asList( + new SchemaAndValueField("val_sf", Schema.OPTIONAL_FLOAT32_SCHEMA, 1.1f), + new SchemaAndValueField("val_f", Schema.OPTIONAL_FLOAT64_SCHEMA, 2.22), + new SchemaAndValueField("val_f_10", Schema.OPTIONAL_FLOAT64_SCHEMA, 3.333), + new SchemaAndValueField("val_r", Schema.OPTIONAL_FLOAT32_SCHEMA, 4.4444f), + new SchemaAndValueField("val_dp", Schema.OPTIONAL_FLOAT64_SCHEMA, 5.55555), + new SchemaAndValueField("val_numeric", Schema.OPTIONAL_FLOAT64_SCHEMA, 1234.567891), + new SchemaAndValueField("val_decimal", Schema.OPTIONAL_FLOAT64_SCHEMA, 1234.567891), + new SchemaAndValueField("val_decimal_vs", Schema.OPTIONAL_FLOAT64_SCHEMA, 77.323), + new SchemaAndValueField("val_decimal_vs2", Schema.OPTIONAL_FLOAT64_SCHEMA, 77.323)); + + private static final List EXPECTED_INT = Arrays.asList( + new SchemaAndValueField("val_int", Schema.OPTIONAL_INT32_SCHEMA, 1), + new SchemaAndValueField("val_int8", Schema.OPTIONAL_INT64_SCHEMA, 22L), + new SchemaAndValueField("val_integer", Schema.OPTIONAL_INT32_SCHEMA, 333), + new SchemaAndValueField("val_smallint", Schema.OPTIONAL_INT16_SCHEMA, (short) 4444), + new SchemaAndValueField("val_bigint", Schema.OPTIONAL_INT64_SCHEMA, 55555L), + new SchemaAndValueField("val_decimal", SpecialValueDecimal.builder(DecimalMode.PRECISE, 10, 0).optional().build(), BigDecimal.valueOf(99999_99999L)), + new SchemaAndValueField("val_numeric", SpecialValueDecimal.builder(DecimalMode.PRECISE, 10, 0).optional().build(), BigDecimal.valueOf(99999_99999L))); + + private static final List EXPECTED_TIME = Arrays.asList( + new SchemaAndValueField("val_date", Date.builder().optional().build(), + (int) LocalDate.of(2024, 3, 27).toEpochDay()), + new SchemaAndValueField("val_time", Time.builder().optional().build(), + LocalTime.of(12, 34, 56).toSecondOfDay() * 1_000), + new SchemaAndValueField("val_datetime", Timestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000), + new SchemaAndValueField("val_timestamp", Timestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000 + 123), + new SchemaAndValueField("val_timestamp_us", MicroTimestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000_000 + 123450)); + // 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 List EXPECTED_TIME_AS_ADAPTIVE = Arrays.asList( + new SchemaAndValueField("val_date", Date.builder().optional().build(), + (int) LocalDate.of(2024, 3, 27).toEpochDay()), + new SchemaAndValueField("val_time", MicroTime.builder().optional().build(), + LocalTime.of(12, 34, 56).toSecondOfDay() * 1_000_000L), + new SchemaAndValueField("val_datetime", Timestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000), + new SchemaAndValueField("val_timestamp", Timestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000 + 123), + new SchemaAndValueField("val_timestamp_us", MicroTimestamp.builder().optional().build(), + LocalDateTime.of(2024, 3, 27, 12, 34, 56).toEpochSecond(ZoneOffset.UTC) * 1_000_000 + 123450)); + // 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 List EXPECTED_TIME_AS_CONNECT = Arrays.asList( + new SchemaAndValueField("val_date", org.apache.kafka.connect.data.Date.builder().optional().build(), + java.util.Date.from(LocalDate.of(2024, 3, 27).atStartOfDay().atOffset(ZoneOffset.UTC).toInstant())), + new SchemaAndValueField("val_time", org.apache.kafka.connect.data.Time.builder().optional().build(), + java.util.Date.from(LocalTime.of(12, 34, 56).atDate(LocalDate.EPOCH).atOffset(ZoneOffset.UTC).toInstant())), + new SchemaAndValueField("val_datetime", org.apache.kafka.connect.data.Timestamp.builder().optional().build(), + java.util.Date.from(LocalDateTime.of(2024, 3, 27, 12, 34, 56).atOffset(ZoneOffset.UTC).toInstant())), + new SchemaAndValueField("val_timestamp", org.apache.kafka.connect.data.Timestamp.builder().optional().build(), + java.util.Date.from(LocalDateTime.of(2024, 3, 27, 12, 34, 56, 123 * 1_000_000).atOffset(ZoneOffset.UTC).toInstant())), + new SchemaAndValueField("val_timestamp_us", org.apache.kafka.connect.data.Timestamp.builder().optional().build(), + java.util.Date.from(LocalDateTime.of(2024, 3, 27, 12, 34, 56, 12345 * 10_000).atOffset(ZoneOffset.UTC).toInstant()))); + // 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_TXT = "TestClob123"; + private static final String CLOB_JSON = Files.readResourceAsString("data/test_lob_data.json"); + + private static final List EXPECTED_CLOB = Arrays.asList( + new SchemaAndValueField("val_clob_inline", Schema.OPTIONAL_STRING_SCHEMA, CLOB_TXT), + new SchemaAndValueField("val_clob_short", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 0, 512)), + new SchemaAndValueField("val_clob_long", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 0, 5000))); + + public static final String CLOB_TXT_UPDATE = "TestClob123Update"; + + private static final List EXPECTED_CLOB_UPDATE = Arrays.asList( + new SchemaAndValueField("val_clob_inline", Schema.OPTIONAL_STRING_SCHEMA, CLOB_TXT_UPDATE), + new SchemaAndValueField("val_clob_short", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 1, 512)), + new SchemaAndValueField("val_clob_long", Schema.OPTIONAL_STRING_SCHEMA, part(CLOB_JSON, 1, 5000))); + + private static final String[] ALL_TABLES = { + "informix.type_string", + "informix.type_fp", + "informix.type_int", + "informix.type_time", + "informix.type_clob" + }; + + private static final String[] ALL_DDLS = { + DDL_STRING, + DDL_FP, + DDL_INT, + DDL_TIME, + DDL_CLOB, + }; + + private static InformixConnection connection; + + @BeforeClass + public static void beforeClass() throws SQLException { + connection = TestHelper.testConnection(); + dropTables(); + } + + @AfterClass + public static void dropTables() throws SQLException { + if (connection != null) { + for (String table : ALL_TABLES) { + TestHelper.dropTable(connection, table); + } + } + } + + protected static void createTables() throws SQLException { + connection.execute(ALL_DDLS); + } + + protected List getAllTables() { + return Arrays.asList(ALL_TABLES); + } + + protected abstract boolean insertRecordsDuringTest(); + + protected abstract Builder connectorConfig(); + + protected abstract void init(TemporalPrecisionMode temporalPrecisionMode) throws Exception; + + @AfterClass + public static void closeConnection() throws SQLException { + if (connection != null) { + connection.close(); + } + } + + @Test + public void stringTypes() throws Exception { + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + insertStringTypes(); + } + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_string"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_STRING); + } + + @Test + public void fpTypes() throws Exception { + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertFpTypes(); + } + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_fp"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_FP); + } + + @Test + @FixFor("DBZ-1552") + public void fpTypesAsString() throws Exception { + stopConnector(); + initializeConnectorTestFramework(); + final Configuration config = connectorConfig() + .with(InformixConnectorConfig.DECIMAL_HANDLING_MODE, DecimalMode.STRING) + .build(); + + start(InformixConnector.class, config); + assertConnectorIsRunning(); + + waitForSnapshotToBeCompleted(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertFpTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_fp"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_FP_AS_STRING); + } + + @Test + @FixFor("DBZ-1552") + public void fpTypesAsDouble() throws Exception { + stopConnector(); + initializeConnectorTestFramework(); + final Configuration config = connectorConfig() + .with(InformixConnectorConfig.DECIMAL_HANDLING_MODE, DecimalMode.DOUBLE) + .build(); + + start(InformixConnector.class, config); + assertConnectorIsRunning(); + + waitForSnapshotToBeCompleted(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertFpTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_fp"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_FP_AS_DOUBLE); + } + + @Test + public void intTypes() throws Exception { + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertIntTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_int"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_INT); + } + + @Test + public void timeTypes() throws Exception { + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertTimeTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_time"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_TIME); + } + + @Test + @FixFor("DBZ-3268") + public void timeTypesAsAdaptiveMicroseconds() throws Exception { + stopConnector(); + init(TemporalPrecisionMode.ADAPTIVE_TIME_MICROSECONDS); + + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertTimeTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_time"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_TIME_AS_ADAPTIVE); + } + + @Test + @FixFor("DBZ-3268") + public void timeTypesAsConnect() throws Exception { + stopConnector(); + init(TemporalPrecisionMode.CONNECT); + + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertTimeTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + final SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_time"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_TIME_AS_CONNECT); + } + + @Test + public void clobTypes() throws Exception { + int expectedRecordCount = 0; + + if (insertRecordsDuringTest()) { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + insertClobTypes(); + } + + waitForAvailableRecords(); + + Testing.debug("Inserted"); + expectedRecordCount++; + + SourceRecords records = consumeRecordsByTopic(expectedRecordCount); + + List testTableRecords = records.recordsForTopic("testdb.informix.type_clob"); + assertThat(testTableRecords).hasSize(expectedRecordCount); + SourceRecord record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + + // insert + if (insertRecordsDuringTest()) { + VerifyRecord.isValidInsert(record, true); + } + else { + VerifyRecord.isValidRead(record); + } + + Struct after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_CLOB); + + if (insertRecordsDuringTest()) { + // Update clob types + updateClobTypes(); + + records = consumeRecordsByTopic(1); + testTableRecords = records.recordsForTopic("testdb.informix.type_clob"); + assertThat(testTableRecords).hasSize(1); + record = testTableRecords.get(0); + + VerifyRecord.isValid(record); + VerifyRecord.isValidUpdate(record, true); + + after = (Struct) ((Struct) record.value()).get("after"); + assertRecord(after, EXPECTED_CLOB_UPDATE); + } + } + + protected static void insertStringTypes() throws SQLException { + connection.execute("INSERT INTO type_string VALUES (0, 'vc', 'nvc', 'lvc', 'c', 'nc');"); + } + + protected static void insertFpTypes() throws SQLException { + connection.execute("INSERT INTO type_fp VALUES (0, 1.1, 2.22, 3.333, 4.4444, 5.55555, 1234.567891, 1234.567891, 77.323, 77.323)"); + } + + protected static void insertIntTypes() throws SQLException { + connection.execute( + "INSERT INTO type_int VALUES (0, 1, 22, 333, 4444, 55555, 9999999999, 9999999999)"); + } + + protected static void insertTimeTypes() throws SQLException { + connection.execute("INSERT INTO type_time VALUES (" + + "0" + + ", '2024-03-27'" + + ", DATETIME(12:34:56) HOUR TO SECOND" + + ", DATETIME(2024-03-27 12:34:56) YEAR TO SECOND" + + ", DATETIME(2024-03-27 12:34:56.123) YEAR TO FRACTION" + + ", DATETIME(2024-03-27 12:34:56.12345) YEAR TO FRACTION(5)" + // + ", '-3-6'" + // + ", '-123 12:34:56'" + + ")"); + } + + protected static void insertClobTypes() throws SQLException { + Clob clob1 = connection.connection().createClob(); + clob1.setString(0, part(CLOB_JSON, 0, 512)); + + Clob clob2 = connection.connection().createClob(); + clob2.setString(0, part(CLOB_JSON, 0, 5000)); + + connection.prepareUpdate("INSERT INTO type_clob VALUES (0, ?, ?, ?)", ps -> { + ps.setString(1, CLOB_TXT); + ps.setClob(2, clob1); + ps.setClob(3, clob2); + }); + connection.commit(); + } + + protected static void updateClobTypes() throws Exception { + Clob clob1 = connection.connection().createClob(); + clob1.setString(0, part(CLOB_JSON, 1, 512)); + + Clob clob2 = connection.connection().createClob(); + clob2.setString(0, part(CLOB_JSON, 1, 5000)); + + connection.prepareUpdate("UPDATE type_clob SET VAL_CLOB_INLINE=?, VAL_CLOB_SHORT=?, VAL_CLOB_LONG=?" + + " WHERE ID = 1", ps -> { + ps.setString(1, CLOB_TXT_UPDATE); + ps.setClob(2, clob1); + ps.setClob(3, clob2); + }); + 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 expected) { + expected.forEach(schemaAndValueField -> schemaAndValueField.assertFor(record)); + } + +} diff --git a/src/test/java/io/debezium/connector/informix/BlockingSnapshotIT.java b/src/test/java/io/debezium/connector/informix/BlockingSnapshotIT.java index 7c45c01..015d8f3 100644 --- a/src/test/java/io/debezium/connector/informix/BlockingSnapshotIT.java +++ b/src/test/java/io/debezium/connector/informix/BlockingSnapshotIT.java @@ -12,35 +12,28 @@ import org.junit.After; import org.junit.Before; import org.junit.Ignore; -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.informix.InformixConnectorConfig.SnapshotMode; import io.debezium.connector.informix.util.TestHelper; import io.debezium.jdbc.JdbcConnection; -import io.debezium.junit.ConditionalFail; import io.debezium.pipeline.AbstractBlockingSnapshotTest; import io.debezium.relational.history.SchemaHistory; public class BlockingSnapshotIT extends AbstractBlockingSnapshotTest { - @Rule - public TestRule conditionalFail = new ConditionalFail(); - private InformixConnection connection; @Before public void before() throws SQLException { connection = TestHelper.testConnection(); + TestHelper.dropTables(connection, "a", "b", "debezium_signal"); connection.execute( - "DROP TABLE IF EXISTS a", - "DROP TABLE IF EXISTS b", - "DROP TABLE IF EXISTS debezium_signal", "CREATE TABLE a (pk int not null, aa int, primary key (pk))", "CREATE TABLE b (pk int not null, aa int, primary key (pk))", - "CREATE TABLE debezium_signal (id varchar(64), type varchar(32), data varchar(255))"); + "CREATE TABLE debezium_signal (id varchar(64), type varchar(32), data lvarchar(2048))"); initializeConnectorTestFramework(); Files.delete(TestHelper.SCHEMA_HISTORY_PATH); Print.disable(); @@ -56,12 +49,20 @@ public void after() throws SQLException { waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); assertConnectorNotRunning(); if (connection != null) { - connection.rollback() - .execute( - "DROP TABLE a", - "DROP TABLE b", - "DROP TABLE debezium_signal") - .close(); + connection.rollback(); + TestHelper.dropTables(connection, "a", "b", "debezium_signal"); + connection.close(); + } + } + + @Override + protected void waitForConnectorToStart() { + super.waitForConnectorToStart(); + try { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + } + catch (InterruptedException e) { + throw new RuntimeException(e); } } @@ -86,33 +87,33 @@ protected List topicNames() { } @Override - protected String tableDataCollectionId() { - return TestHelper.TEST_DATABASE + '.' + tableName(); + protected String tableName() { + return "informix.a"; } @Override - protected List tableDataCollectionIds() { - return tableNames().stream().map(name -> TestHelper.TEST_DATABASE + '.' + name).collect(Collectors.toList()); + protected List tableNames() { + return List.of(tableName(), "informix.b"); } @Override - protected String tableName() { - return "informix.a"; + protected String tableDataCollectionId() { + return TestHelper.TEST_DATABASE + '.' + tableName(); } @Override - protected List tableNames() { - return List.of(tableName(), "informix.b"); + protected String escapedTableDataCollectionId() { + return "\\\"" + TestHelper.TEST_DATABASE + "\\\".\\\"informix\\\".\\\"a\\\""; } @Override - protected String signalTableName() { - return "informix.debezium_signal"; + protected List tableDataCollectionIds() { + return tableNames().stream().map(name -> TestHelper.TEST_DATABASE + '.' + name).collect(Collectors.toList()); } @Override - protected String escapedTableDataCollectionId() { - return "\\\"testdb\\\".\\\"informix\\\".\\\"a\\\""; + protected String signalTableName() { + return "informix.debezium_signal"; } @Override @@ -126,6 +127,8 @@ protected Builder config() { .with(InformixConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) .with(InformixConnectorConfig.SIGNAL_DATA_COLLECTION, this::signalTableNameSanitized) .with(InformixConnectorConfig.SNAPSHOT_MODE_TABLES, this::tableDataCollectionId) + .with(InformixConnectorConfig.STORE_ONLY_CAPTURED_TABLES_DDL, true) + .with(InformixConnectorConfig.INCLUDE_SCHEMA_CHANGES, false) .with(InformixConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 100) .with(InformixConnectorConfig.CDC_BUFFERSIZE, 0x800); } @@ -136,6 +139,12 @@ protected Builder mutableConfig(boolean signalTableOnly, boolean storeOnlyCaptur .with(SchemaHistory.STORE_ONLY_CAPTURED_TABLES_DDL, storeOnlyCapturedDdl); } + @Override + protected Configuration.Builder historizedMutableConfig(boolean signalTableOnly, boolean storeOnlyCapturedDdl) { + return mutableConfig(signalTableOnly, storeOnlyCapturedDdl) + .with(InformixConnectorConfig.INCLUDE_SCHEMA_CHANGES, true); + } + @Override protected String connector() { return TestHelper.TEST_CONNECTOR; diff --git a/src/test/java/io/debezium/connector/informix/CustomTestSnapshot.java b/src/test/java/io/debezium/connector/informix/CustomTestSnapshot.java index 075ecf1..23acef9 100644 --- a/src/test/java/io/debezium/connector/informix/CustomTestSnapshot.java +++ b/src/test/java/io/debezium/connector/informix/CustomTestSnapshot.java @@ -17,7 +17,7 @@ import io.debezium.spi.snapshot.Snapshotter; /** - * This is a small class used in PostgresConnectorIT to test a custom snapshot + * This is a small class used in CustomSnapshotterIT to test a custom snapshot * * It is tightly coupled to the test there, but needs to be placed here in order * to allow for class loading to work @@ -34,10 +34,8 @@ public String name() { @Override public void injectBeanRegistry(BeanRegistry beanRegistry) { - Offsets db2OffsetContextOffsets = beanRegistry.lookupByName(StandardBeanNames.OFFSETS, Offsets.class); - for (InformixOffsetContext offset : db2OffsetContextOffsets.getOffsets().values()) { - hasState = offset != null; - } + Offsets informixOffsets = beanRegistry.lookupByName(StandardBeanNames.OFFSETS, Offsets.class); + hasState = informixOffsets.getTheOnlyOffset() != null; } @Override diff --git a/src/test/java/io/debezium/connector/informix/FieldTest.java b/src/test/java/io/debezium/connector/informix/FieldTest.java new file mode 100644 index 0000000..b919b0d --- /dev/null +++ b/src/test/java/io/debezium/connector/informix/FieldTest.java @@ -0,0 +1,19 @@ +/* + * 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.informix; + +import org.junit.Before; + +import io.debezium.config.AbstractFieldTest; + +public class FieldTest extends AbstractFieldTest { + + @Before + public void before() { + setAllConnectorFields(InformixConnectorConfig.ALL_FIELDS); + } + +} diff --git a/src/test/java/io/debezium/connector/informix/IncrementalSnapshotIT.java b/src/test/java/io/debezium/connector/informix/IncrementalSnapshotIT.java index 2995c73..48a8cf0 100644 --- a/src/test/java/io/debezium/connector/informix/IncrementalSnapshotIT.java +++ b/src/test/java/io/debezium/connector/informix/IncrementalSnapshotIT.java @@ -19,8 +19,6 @@ import io.debezium.connector.informix.util.TestHelper; import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotTest; -import io.debezium.relational.RelationalDatabaseConnectorConfig; -import io.debezium.relational.history.SchemaHistory; public class IncrementalSnapshotIT extends AbstractIncrementalSnapshotTest { @@ -29,11 +27,8 @@ public class IncrementalSnapshotIT extends AbstractIncrementalSnapshotTest tableDataCollectionIds() { - return tableNames().stream().map(name -> TestHelper.TEST_DATABASE + '.' + name).collect(Collectors.toList()); - } - - protected String tableIncludeList() { - return String.join(",", tableDataCollectionIds()); - } - @Override protected String tableName() { return "informix.a"; @@ -117,6 +105,16 @@ protected String noPKTableName() { return "informix.c"; } + @Override + protected String tableDataCollectionId() { + return TestHelper.TEST_DATABASE + '.' + tableName(); + } + + @Override + protected List tableDataCollectionIds() { + return tableNames().stream().map(name -> TestHelper.TEST_DATABASE + '.' + name).collect(Collectors.toList()); + } + @Override protected String noPKTableDataCollectionId() { return TestHelper.TEST_DATABASE + "." + noPKTableName(); @@ -132,26 +130,37 @@ protected String signalTableNameSanitized() { return TestHelper.TEST_DATABASE + '.' + signalTableName(); } + protected String tableIncludeList() { + return String.join(",", tableDataCollectionIds()); + } + @Override protected Builder config() { return TestHelper.defaultConfig() .with(InformixConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NO_DATA) .with(InformixConnectorConfig.SIGNAL_DATA_COLLECTION, this::signalTableNameSanitized) - .with(InformixConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 250) - .with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, noPKTableDataCollectionId() + ":pk1,pk2,pk3,pk4"); + .with(InformixConnectorConfig.MSG_KEY_COLUMNS, noPKTableDataCollectionId() + ":pk1,pk2,pk3,pk4") + .with(InformixConnectorConfig.STORE_ONLY_CAPTURED_TABLES_DDL, true) + .with(InformixConnectorConfig.INCLUDE_SCHEMA_CHANGES, false) + .with(InformixConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 200) + .with(InformixConnectorConfig.CDC_BUFFERSIZE, 0x800); } @Override protected Builder mutableConfig(boolean signalTableOnly, boolean storeOnlyCapturedDdl) { Builder config = config() .with(InformixConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) - .with(SchemaHistory.STORE_ONLY_CAPTURED_TABLES_DDL, storeOnlyCapturedDdl) - .with(RelationalDatabaseConnectorConfig.MSG_KEY_COLUMNS, noPKTableDataCollectionId() + ":pk1,pk2,pk3,pk4"); + .with(InformixConnectorConfig.STORE_ONLY_CAPTURED_TABLES_DDL, storeOnlyCapturedDdl); return signalTableOnly ? config.with(InformixConnectorConfig.TABLE_EXCLUDE_LIST, this::tableDataCollectionId) : config.with(InformixConnectorConfig.TABLE_INCLUDE_LIST, this::tableIncludeList); } + @Override + protected int defaultIncrementalSnapshotChunkSize() { + return 10; + } + @Override protected String connector() { return TestHelper.TEST_CONNECTOR; diff --git a/src/test/java/io/debezium/connector/informix/InformixCdcTypesIT.java b/src/test/java/io/debezium/connector/informix/InformixCdcTypesIT.java deleted file mode 100644 index cad4b9e..0000000 --- a/src/test/java/io/debezium/connector/informix/InformixCdcTypesIT.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * 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.informix; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.sql.SQLException; -import java.time.LocalDate; -import java.time.temporal.ChronoUnit; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; - -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.source.SourceRecord; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import io.debezium.config.Configuration; -import io.debezium.connector.informix.InformixConnectorConfig.SnapshotMode; -import io.debezium.connector.informix.util.TestHelper; -import io.debezium.data.SourceRecordAssert; -import io.debezium.embedded.async.AbstractAsyncEngineConnectorTest; -import io.debezium.relational.RelationalDatabaseConnectorConfig; -import io.debezium.relational.RelationalDatabaseConnectorConfig.DecimalHandlingMode; -import io.debezium.time.Date; -import io.debezium.util.Testing; - -public class InformixCdcTypesIT extends AbstractAsyncEngineConnectorTest { - - private InformixConnection connection; - - @Before - public void before() throws SQLException { - connection = TestHelper.testConnection(); - connection.execute( - "drop table if exists test_bigint", - "drop table if exists test_bigserial", - "drop table if exists test_char", - "drop table if exists test_date", - "drop table if exists test_decimal", - "drop table if exists test_decimal_20", - "drop table if exists test_decimal_20_5", - "create table test_bigint(a bigint)", - "create table test_bigserial(a bigserial)", - "create table test_char(a char)", - "create table test_date(a date)", - "create table test_decimal(a decimal)", - "create table test_decimal_20(a decimal(20))", - "create table test_decimal_20_5(a decimal(20, 5))"); - - initializeConnectorTestFramework(); - Testing.Files.delete(TestHelper.SCHEMA_HISTORY_PATH); - Testing.Print.enable(); - } - - @After - public void after() throws SQLException { - /* - * Since all DDL operations are forbidden during Informix CDC, - * we have to ensure the connector is properly shut down before dropping tables. - */ - stopConnector(); - waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); - assertConnectorNotRunning(); - if (connection != null) { - connection.rollback() - .execute("drop table test_bigint", - "drop table test_bigserial", - "drop table test_char", - "drop table test_date", - "drop table test_decimal", - "drop table test_decimal_20", - "drop table test_decimal_20_5") - .close(); - } - } - - @Test - public void testTypes() throws Exception { - - final Configuration config = TestHelper.defaultConfig() - .with(InformixConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NO_DATA) - .with(RelationalDatabaseConnectorConfig.DECIMAL_HANDLING_MODE, DecimalHandlingMode.STRING) - .build(); - - start(InformixConnector.class, config); - - assertConnectorIsRunning(); - - waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); - - /* - * bigint - */ - Long testLongValue = new Random().nextLong(); - insertOneAndValidate("test_bigint", Schema.OPTIONAL_INT64_SCHEMA, testLongValue.toString(), testLongValue); - - /* - * bigserial - */ - Long testBigSerialValue = new Random().nextLong(); - insertOneAndValidate("test_bigserial", Schema.INT64_SCHEMA, testBigSerialValue.toString(), testBigSerialValue); - - /* - * char - */ - // insertOneAndValidate("test_char", Schema.OPTIONAL_STRING_SCHEMA, "'a'", 'a'); - - /* - * date - * - * As described from official manual: - * "The DATE data type stores the calendar date. DATE data types require four bytes. A - * calendar date is stored internally as an integer value equal to the number of days - * since December 31, 1899." - * - https://www.ibm.com/docs/en/informix-servers/12.10?topic=types-date-data-type - * - * TODO: But, as we test locally, it seems the base date is "1970-01-01", not the "1899-12-31". - */ - String[] arrTestDate = new String[]{ "2022-01-01" }; - for (String strTestDate : arrTestDate) { - Integer d = Math.toIntExact(diffInDays("1970-01-01", strTestDate)); - insertOneAndValidate("test_date", Date.builder().optional().build(), "'" + strTestDate + "'", d); - } - - /* - * decimal - */ - Map decimal_data_expect = new LinkedHashMap<>() { - { - put("12.1", "12.1"); - put("22.12345678901234567890", "22.12345678901235"); // Rounded number - put("12345678901234567890.12345", "12345678901234570000"); - } - }; - for (Map.Entry entry : decimal_data_expect.entrySet()) { - insertOneAndValidate("test_decimal", Schema.OPTIONAL_STRING_SCHEMA, entry.getKey(), entry.getValue()); - } - - /* - * decimal(20) - */ - Map decimal_20_data_expect = new LinkedHashMap<>() { - { - put("88.07", "88.07"); - put("33.12345", "33.12345"); // Rounded number - put("123456789012345.12345", "123456789012345.12345"); - } - }; - for (Map.Entry entry : decimal_20_data_expect.entrySet()) { - insertOneAndValidate("test_decimal_20", Schema.OPTIONAL_STRING_SCHEMA, entry.getKey(), entry.getValue()); - } - - /* - * decimal(20, 5) - */ - Map decimal_20_5_data_expect = new LinkedHashMap<>() { - { - put("12.1", "12.10000"); - put("22.12345", "22.12345"); // Rounded number - put("123456789012345.12345", "123456789012345.12345"); - } - }; - for (Map.Entry entry : decimal_20_5_data_expect.entrySet()) { - insertOneAndValidate("test_decimal_20_5", Schema.OPTIONAL_STRING_SCHEMA, entry.getKey(), entry.getValue()); - } - } - - private void insertOneAndValidate(String tableName, Schema valueSchema, String insertValue, Object expectValue) throws Exception { - String topicName = String.format("testdb.informix.%s", tableName); - connection.execute(String.format("insert into %s values(%s)", tableName, insertValue)); - - waitForAvailableRecords(); - - List records = consumeRecordsByTopic(1).recordsForTopic(topicName); - assertThat(records).isNotNull().hasSize(1); - - Schema aSchema = SchemaBuilder.struct() - .optional().name(String.format("%s.Value", topicName)).field("a", valueSchema) - .build(); - Struct aStruct = new Struct(aSchema).put("a", expectValue); - - SourceRecordAssert.assertThat(records.get(0)).valueAfterFieldIsEqualTo(aStruct); - } - - private long diffInDays(String one, String other) { - return LocalDate.parse(one).until(LocalDate.parse(other), ChronoUnit.DAYS); - } - -} diff --git a/src/test/java/io/debezium/connector/informix/InformixConnectionIT.java b/src/test/java/io/debezium/connector/informix/InformixConnectionIT.java index c6c52ce..774878b 100644 --- a/src/test/java/io/debezium/connector/informix/InformixConnectionIT.java +++ b/src/test/java/io/debezium/connector/informix/InformixConnectionIT.java @@ -8,11 +8,12 @@ import org.junit.Test; import io.debezium.connector.informix.util.TestHelper; +import io.debezium.util.Testing; /** * Integration test for {@link InformixConnection} */ -public class InformixConnectionIT { +public class InformixConnectionIT implements Testing { @Test public void shouldEnableDatabaseLogging() throws Exception { diff --git a/src/test/java/io/debezium/connector/informix/NotificationsIT.java b/src/test/java/io/debezium/connector/informix/NotificationsIT.java index c607eae..2ec658b 100644 --- a/src/test/java/io/debezium/connector/informix/NotificationsIT.java +++ b/src/test/java/io/debezium/connector/informix/NotificationsIT.java @@ -23,9 +23,9 @@ public class NotificationsIT extends AbstractNotificationsIT @Before public void before() throws SQLException { connection = TestHelper.testConnection(); - connection.execute( - "DROP TABLE IF EXISTS tablea", - "CREATE TABLE tablea (id int not null, cola varchar(30), primary key(id))"); + + TestHelper.dropTable(connection, "tablea"); + connection.execute("CREATE TABLE tablea (id int not null, cola varchar(30), primary key(id))"); initializeConnectorTestFramework(); Files.delete(TestHelper.SCHEMA_HISTORY_PATH); @@ -42,10 +42,9 @@ public void after() throws SQLException { waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); assertConnectorNotRunning(); if (connection != null) { - connection.rollback() - .execute( - "DROP TABLE tablea") - .close(); + connection.rollback(); + TestHelper.dropTable(connection, "tablea"); + connection.close(); } } @@ -74,6 +73,7 @@ protected String snapshotStatusResult() { return "COMPLETED"; } + @Override protected List collections() { return List.of("testdb.informix.tablea"); } diff --git a/src/test/java/io/debezium/connector/informix/OutboxEventRouterIT.java b/src/test/java/io/debezium/connector/informix/OutboxEventRouterIT.java new file mode 100644 index 0000000..4c19a7f --- /dev/null +++ b/src/test/java/io/debezium/connector/informix/OutboxEventRouterIT.java @@ -0,0 +1,185 @@ +/* + * 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.informix; + +import org.apache.kafka.connect.data.Schema; +import org.junit.After; +import org.junit.Before; + +import io.debezium.config.Configuration; +import io.debezium.connector.informix.InformixConnectorConfig.SnapshotMode; +import io.debezium.connector.informix.util.TestHelper; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.transforms.outbox.AbstractEventRouterTest; +import io.debezium.transforms.outbox.EventRouter; + +/** + * An integration test for Informix and the {@link EventRouter} for outbox. + * + * @author Chris Cranford, Lars M Johansson + */ +public class OutboxEventRouterIT extends AbstractEventRouterTest { + + private static final String SETUP_OUTBOX_TABLE = "CREATE TABLE outbox (" + + "id varchar(64) not null primary key, " + + "aggregatetype varchar(255) not null, " + + "aggregateid varchar(255) not null, " + + "type varchar(255) not null, " + + "payload lvarchar(4000))"; + + private InformixConnection connection; + + @Before + @Override + public void beforeEach() throws Exception { + connection = TestHelper.testConnection(); + + initializeConnectorTestFramework(); + Files.delete(TestHelper.SCHEMA_HISTORY_PATH); + + super.beforeEach(); + } + + @After + @Override + public void afterEach() throws Exception { + super.afterEach(); + waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + assertConnectorNotRunning(); + if (connection != null && connection.isConnected()) { + connection.rollback(); + TestHelper.dropTable(connection, tableName()); + connection.close(); + } + } + + @Override + protected Class getConnectorClass() { + return InformixConnector.class; + } + + @Override + protected JdbcConnection databaseConnection() { + return connection; + } + + @Override + protected Configuration.Builder getConfigurationBuilder(boolean initialSnapshot) { + final SnapshotMode snapshotMode = initialSnapshot ? SnapshotMode.INITIAL : SnapshotMode.NO_DATA; + return TestHelper.defaultConfig() + .with(InformixConnectorConfig.SNAPSHOT_MODE, snapshotMode.getValue()) + .with(InformixConnectorConfig.TABLE_INCLUDE_LIST, TestHelper.TEST_DATABASE + "." + tableName()); + } + + @Override + protected String getSchemaNamePrefix() { + return "testdb.informix.outbox."; + } + + @Override + protected Schema getPayloadSchema() { + return Schema.OPTIONAL_STRING_SCHEMA; + } + + @Override + protected String tableName() { + return "informix.outbox"; + } + + @Override + protected String topicName() { + return TestHelper.TEST_DATABASE + ".informix.outbox"; + } + + @Override + protected void createTable() throws Exception { + TestHelper.dropTable(connection, tableName()); + connection.execute(SETUP_OUTBOX_TABLE); + } + + @Override + protected String createInsert(String eventId, + String eventType, + String aggregateType, + String aggregateId, + String payloadJson, + String additional) { + StringBuilder insert = new StringBuilder(); + insert.append("INSERT INTO informix.outbox VALUES ("); + insert.append("'").append(eventId).append("', "); + insert.append("'").append(aggregateType).append("', "); + insert.append("'").append(aggregateId).append("', "); + insert.append("'").append(eventType).append("', "); + if (payloadJson != null) { + insert.append("'").append(payloadJson).append("'"); + } + else { + insert.append("NULL"); + } + if (additional != null) { + insert.append(additional); + } + insert.append(")"); + return insert.toString(); + } + + @Override + protected void waitForSnapshotCompleted() throws InterruptedException { + waitForSnapshotToBeCompleted(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + } + + @Override + protected void waitForStreamingStarted() throws InterruptedException { + waitForStreamingRunning(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + waitForAvailableRecords(); + } + + @Override + protected void alterTableWithExtra4Fields() throws Exception { + stopConnector(); + waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + + connection.execute("ALTER TABLE informix.outbox add version integer not null"); + connection.execute("ALTER TABLE informix.outbox add somebooltype boolean not null"); + connection.execute("ALTER TABLE informix.outbox add createdat datetime year to fraction not null"); + connection.execute("ALTER TABLE informix.outbox add is_deleted boolean default 'f'"); + + startConnectorWithNoSnapshot(); + } + + @Override + protected void alterTableWithTimestampField() throws Exception { + stopConnector(); + waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + connection.execute("ALTER TABLE informix.outbox add createdat datetime year to fraction not null"); + startConnectorWithNoSnapshot(); + } + + @Override + protected void alterTableModifyPayload() throws Exception { + stopConnector(); + waitForConnectorShutdown(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + connection.execute("ALTER TABLE informix.outbox modify (payload lvarchar(1000))"); + startConnectorWithNoSnapshot(); + } + + @Override + protected String getAdditionalFieldValues(boolean deleted) { + return ", 1, 't', DATETIME(2019-03-24 20:52:59) YEAR TO FRACTION, " + (deleted ? "'t'" : "'f'"); + } + + @Override + protected String getAdditionalFieldValuesTimestampOnly() { + return ", DATETIME(2019-03-24 20:52:59) YEAR TO FRACTION"; + } + + private void startConnectorWithNoSnapshot() throws Exception { + start(getConnectorClass(), getConfigurationBuilder(false).build()); + assertConnectorIsRunning(); + waitForStreamingStarted(); + assertNoRecordsToConsume(); + } +} diff --git a/src/test/java/io/debezium/connector/informix/SnapshotDatatypesIT.java b/src/test/java/io/debezium/connector/informix/SnapshotDatatypesIT.java new file mode 100644 index 0000000..fe7ff8d --- /dev/null +++ b/src/test/java/io/debezium/connector/informix/SnapshotDatatypesIT.java @@ -0,0 +1,94 @@ +/* + * 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.informix; + +import java.sql.SQLException; + +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; + +import io.debezium.config.Configuration; +import io.debezium.config.Configuration.Builder; +import io.debezium.connector.informix.util.TestHelper; +import io.debezium.jdbc.TemporalPrecisionMode; +import io.debezium.util.Testing; + +/** + * Integration test to verify different Oracle datatypes as captured during initial snapshotting. + * + * @author Jiri Pechanec, Lars M Johansson + */ +public class SnapshotDatatypesIT extends AbstractInformixDatatypesTest { + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void beforeClass() throws SQLException { + AbstractInformixDatatypesTest.beforeClass(); + createTables(); + + insertStringTypes(); + insertFpTypes(); + insertIntTypes(); + insertTimeTypes(); + insertClobTypes(); + } + + @Before + public void before() throws Exception { + init(TemporalPrecisionMode.ADAPTIVE); + } + + @Override + protected void init(TemporalPrecisionMode temporalPrecisionMode) throws Exception { + initializeConnectorTestFramework(); + Testing.Debug.enable(); + Testing.Files.delete(TestHelper.SCHEMA_HISTORY_PATH); + + Configuration config = connectorConfig() + .with(InformixConnectorConfig.TIME_PRECISION_MODE, temporalPrecisionMode) + .build(); + + start(InformixConnector.class, config); + assertConnectorIsRunning(); + + waitForSnapshotToBeCompleted(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + } + + protected Builder connectorConfig() { + return TestHelper.defaultConfig() + .with(InformixConnectorConfig.TABLE_INCLUDE_LIST, getTableIncludeList()); + } + + private String getTableIncludeList() { + switch (name.getMethodName()) { + case "stringTypes": + return "testdb.informix.type_string"; + case "fpTypes": + case "fpTypesAsString": + case "fpTypesAsDouble": + return "testdb.informix.type_fp"; + case "intTypes": + return "testdb.informix.type_int"; + case "timeTypes": + case "timeTypesAsAdaptiveMicroseconds": + case "timeTypesAsConnect": + return "testdb.informix.type_time"; + case "clobTypes": + return "testdb.informix.type_clob"; + default: + throw new IllegalArgumentException("Unexpected test method: " + name.getMethodName()); + } + } + + @Override + protected boolean insertRecordsDuringTest() { + return false; + } +} diff --git a/src/test/java/io/debezium/connector/informix/StreamingDatatypesIT.java b/src/test/java/io/debezium/connector/informix/StreamingDatatypesIT.java new file mode 100644 index 0000000..8062d7d --- /dev/null +++ b/src/test/java/io/debezium/connector/informix/StreamingDatatypesIT.java @@ -0,0 +1,63 @@ +/* + * 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.informix; + +import java.util.stream.Collectors; + +import org.junit.Before; + +import io.debezium.config.Configuration; +import io.debezium.config.Configuration.Builder; +import io.debezium.connector.informix.InformixConnectorConfig.SnapshotMode; +import io.debezium.connector.informix.util.TestHelper; +import io.debezium.jdbc.TemporalPrecisionMode; +import io.debezium.util.Testing; + +/** + * Integration test to verify different Oracle datatypes as captured during streaming. + * + * @author Jiri Pechanec, Lars M Johansson + */ +public class StreamingDatatypesIT extends AbstractInformixDatatypesTest { + + @Before + public void before() throws Exception { + init(TemporalPrecisionMode.ADAPTIVE); + } + + @Override + protected void init(TemporalPrecisionMode temporalPrecisionMode) throws Exception { + dropTables(); + initializeConnectorTestFramework(); + Testing.Files.delete(TestHelper.SCHEMA_HISTORY_PATH); + + Configuration config = connectorConfig() + .with(InformixConnectorConfig.TIME_PRECISION_MODE, temporalPrecisionMode) + .build(); + + createTables(); + + start(InformixConnector.class, config); + assertConnectorIsRunning(); + + waitForSnapshotToBeCompleted(TestHelper.TEST_CONNECTOR, TestHelper.TEST_DATABASE); + } + + protected Builder connectorConfig() { + String tableIncludeList = getAllTables().stream() + .map(t -> TestHelper.TEST_DATABASE + '.' + t) + .collect(Collectors.joining(",")); + + return TestHelper.defaultConfig() + .with(InformixConnectorConfig.TABLE_INCLUDE_LIST, tableIncludeList) + .with(InformixConnectorConfig.SNAPSHOT_MODE, SnapshotMode.NO_DATA); + } + + @Override + protected boolean insertRecordsDuringTest() { + return true; + } +} diff --git a/src/test/java/io/debezium/connector/informix/TransactionMetadataIT.java b/src/test/java/io/debezium/connector/informix/TransactionMetadataIT.java index 8ebee75..0481226 100644 --- a/src/test/java/io/debezium/connector/informix/TransactionMetadataIT.java +++ b/src/test/java/io/debezium/connector/informix/TransactionMetadataIT.java @@ -123,7 +123,7 @@ public void transactionMetadata() throws Exception { protected String assertBeginTransaction(SourceRecord record) { final Struct begin = (Struct) record.value(); final Struct beginKey = (Struct) record.key(); - final Map offset = (Map) record.sourceOffset(); + final Map offset = record.sourceOffset(); assertThat(begin.getString("status")).isEqualTo("BEGIN"); assertThat(begin.getInt64("event_count")).isNull(); @@ -139,7 +139,7 @@ protected String assertBeginTransaction(SourceRecord record) { protected void assertEndTransaction(SourceRecord record, String beginTxId, long expectedEventCount, Map expectedPerTableCount) { final Struct end = (Struct) record.value(); final Struct endKey = (Struct) record.key(); - final Map offset = (Map) record.sourceOffset(); + final Map offset = record.sourceOffset(); final String expectedId = Arrays.stream(beginTxId.split(":")).findFirst().get(); final String expectedTxId = String.format("%s:%s", expectedId, offset.get("commit_lsn")); @@ -157,7 +157,7 @@ protected void assertEndTransaction(SourceRecord record, String beginTxId, long @Override protected void assertRecordTransactionMetadata(SourceRecord record, String beginTxId, long expectedTotalOrder, long expectedCollectionOrder) { final Struct change = ((Struct) record.value()).getStruct("transaction"); - final Map offset = (Map) record.sourceOffset(); + final Map offset = record.sourceOffset(); final String expectedId = Arrays.stream(beginTxId.split(":")).findFirst().get(); final String expectedTxId = String.format("%s:%s", expectedId, offset.get("commit_lsn")); diff --git a/src/test/java/io/debezium/connector/informix/util/TestHelper.java b/src/test/java/io/debezium/connector/informix/util/TestHelper.java index 61d058e..2270a80 100644 --- a/src/test/java/io/debezium/connector/informix/util/TestHelper.java +++ b/src/test/java/io/debezium/connector/informix/util/TestHelper.java @@ -95,6 +95,16 @@ public static InformixConnection adminConnection() { return new InformixConnection(TestHelper.adminJdbcConfig()); } + public static void dropTable(InformixConnection connection, String table) throws SQLException { + connection.execute("drop table if exists " + table); + } + + public static void dropTables(InformixConnection connection, String... tables) throws SQLException { + for (String table : tables) { + dropTable(connection, table); + } + } + private static class LazyConnectionHolder { static final InformixConnection INSTANCE = new InformixConnection(TestHelper.defaultJdbcConfig()); } diff --git a/src/test/resources/data/test_lob_data.json b/src/test/resources/data/test_lob_data.json new file mode 100644 index 0000000..eaa6b60 --- /dev/null +++ b/src/test/resources/data/test_lob_data.json @@ -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" + } +] \ No newline at end of file