diff --git a/java/yb-cdc/pom.xml b/java/yb-cdc/pom.xml index 4e547872eb8c..ddfbde55a0e9 100644 --- a/java/yb-cdc/pom.xml +++ b/java/yb-cdc/pom.xml @@ -28,16 +28,17 @@ org.apache.kafka kafka-clients - 2.8.1 + 2.3.0 org.apache.avro avro - 1.10.2 + 1.9.0 commons-io commons-io + 2.5 org.yb @@ -69,10 +70,12 @@ commons-cli commons-cli + 1.2 commons-codec commons-codec + 1.10 org.apache.commons @@ -83,6 +86,33 @@ gson 2.8.0 + + org.postgresql + postgresql + 42.2.23 + + + org.mybatis + mybatis + 3.4.5 + + + + com.yugabyte + java-driver-core + 4.6.0-yb-6 + + + + com.yugabyte + jdbc-yugabytedb + 42.3.0-beta.1 + + + ${junit.groupId} + junit + test + @@ -114,6 +144,7 @@ org.apache.maven.plugins maven-dependency-plugin + 3.2.0 copy-dependencies @@ -134,13 +165,14 @@ org.apache.maven.plugins maven-assembly-plugin + 3.3.0 yb-cdc-connector false true - org.yb.cdc.Main + org.yb.cdc.CDCConsoleSubscriber @@ -166,6 +198,14 @@ clean verify + + maven-surefire-plugin + 2.22.1 + + + maven-jar-plugin + 3.0.2 + diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/CDCConsoleSubscriber.java b/java/yb-cdc/src/main/java/org/yb/cdc/CDCConsoleSubscriber.java new file mode 100644 index 000000000000..27d7bd184439 --- /dev/null +++ b/java/yb-cdc/src/main/java/org/yb/cdc/CDCConsoleSubscriber.java @@ -0,0 +1,58 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.*; + +public class CDCConsoleSubscriber { + private static final Logger LOG = Logger.getLogger(CDCConsoleSubscriber.class); + + private ConcurrentLogConnector connector; + + public CDCConsoleSubscriber(CmdLineOpts cmdLineOpts, OutputClient opClient) throws Exception { + connector = new ConcurrentLogConnector(cmdLineOpts, opClient); + } + + public void run() { + try { + connector.run(); + } catch (Exception e) { + e.printStackTrace(); + LOG.error("Application ran into an error: ", e); + System.exit(0); + } + } + + public void close() { + try { + connector.close(); + } catch (Exception e) { + System.exit(0); + } + } + + public static void main(String[] args) throws Exception { + LOG.info("Starting CDC Console Connector..."); + + CmdLineOpts configuration = CmdLineOpts.createFromArgs(args); + try { + CDCConsoleSubscriber subscriber = new CDCConsoleSubscriber(configuration, new LogClient()); + subscriber.run(); + } + catch (Exception e) { + e.printStackTrace(); + System.exit(1); + } + } +} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/CmdLineOpts.java b/java/yb-cdc/src/main/java/org/yb/cdc/CmdLineOpts.java index f09626ea2e2d..85fa2e970b18 100644 --- a/java/yb-cdc/src/main/java/org/yb/cdc/CmdLineOpts.java +++ b/java/yb-cdc/src/main/java/org/yb/cdc/CmdLineOpts.java @@ -13,77 +13,299 @@ package org.yb.cdc; -import org.apache.commons.cli.BasicParser; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.*; +import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.log4j.Logger; +import org.yb.client.AsyncYBClient; public class CmdLineOpts { - - private static final String DEFAULT_MASTER_ADDRS = "127.0.0.1:7100"; + private static final String DEFAULT_MASTER_ADDRESS = "127.0.0.1:7100"; private static final Logger LOG = Logger.getLogger(CmdLineOpts.class); public String tableName; public String namespaceName; - public String masterAddrs = DEFAULT_MASTER_ADDRS; + public String masterAddrs = DEFAULT_MASTER_ADDRESS; public String streamId = ""; + public boolean enableSnapshot; + public String sslCertFile; + public String clientCertFile; + public String clientKeyFile; + public int maxTablets = AsyncYBClient.DEFAULT_MAX_TABLETS; - public static CmdLineOpts createFromArgs(String[] args) throws Exception { - Options options = new Options(); + // Config file path to be provided from command line. + public String configFile = ""; - options.addOption("master_addrs", true, "List of YB master ips to contact"); - options.addOption("table_name", true, - "Table to get change capture from in format ."); - options.addOption("stream_id", true, - "Optional stream ID. Use this if you already have a CDC stream set up on the table"); + // This option will not go with the help message. Strictly for internal testing purposes. + public int pollingInterval = 200; - // Do the actual arg parsing. - CommandLineParser parser = new BasicParser(); - CommandLine commandLine = null; + String lineSeparator = System.lineSeparator(); + String configFileHelpMessage = "\nCreate a config.properties file with the" + + " following parameters and " + + "pass its path in config" + .concat(lineSeparator) + .concat("\tnum.io.threads=") + .concat(lineSeparator) + .concat("\tschema.name=") + .concat(lineSeparator) + .concat("\ttable.name=") + .concat(lineSeparator) + .concat("\tsocket.read.timeout.ms=" + + "") + .concat(lineSeparator) + .concat("\toperation.timeout.ms=" + + "") + .concat(lineSeparator) + .concat("\tadmin.operation.timeout.ms=" + + "") + .concat(lineSeparator) + .concat("\tmaster.addrs=") + .concat(lineSeparator) + .concat("\tstream.id=" + lineSeparator + + "\t (if you want one to be created automatically, " + + "leave this empty)") + .concat(lineSeparator) + .concat(lineSeparator) + .concat(lineSeparator) + .concat("If you will provide stream_id, master_address, table_name " + + "from command line, it would overwrite the ones provided via " + + "config.properties file") + .concat(lineSeparator); - try { - commandLine = parser.parse(options, args); - } catch (ParseException e) { - LOG.error("Error in args, use the --help option to see usage. Exception:", e); - System.exit(0); - } + String helpMessage = lineSeparator + .concat("yb-cdc-connector.jar is a tool to use the CDC Client from console") + .concat(lineSeparator) + .concat("Usage: java -jar yb-cdc-connector.jar" + + " --config_file ") + .concat(" []").concat(lineSeparator) + .concat("Options:").concat(lineSeparator) + .concat(" --help").concat(lineSeparator) + .concat(" Show help").concat(lineSeparator) + .concat(" --config_file").concat(lineSeparator) + .concat(" Provide a path to custom config file").concat(lineSeparator) + .concat(" --show_config_help").concat(lineSeparator) + .concat(" Show the parameters that should be there in custom config.properties file") + .concat(lineSeparator) + .concat(" --master_address").concat(lineSeparator) + .concat(" List of YB Master IPs to contact").concat(lineSeparator) + .concat(" --table_name").concat(lineSeparator) + .concat(" Table to capture change from, in format .
or " + + "..
") + .concat(lineSeparator) + .concat(" --stream_id").concat(lineSeparator) + .concat(" Optional DB stream ID, provide this if you have" + + " already a CDC stream setup") + .concat(" on a table").concat(lineSeparator) + .concat(" --create_new_db_stream_id").concat(lineSeparator) + .concat(" Flag to specify that a new DB stream ID is supposed to be created for use") + .concat(lineSeparator) + .concat(" --disable_snapshot").concat(lineSeparator) + .concat(" Flag to specify whether to disable snapshot, default behaviour is to " + + "take snapshot") + .concat(lineSeparator) + .concat(" --ssl_cert_file").concat(lineSeparator) + .concat(" Path to SSL certificate file") + .concat(lineSeparator) + .concat(" --client_cert_file").concat(lineSeparator) + .concat(" Path to client certificate file") + .concat(lineSeparator) + .concat(" --client_key_file").concat(lineSeparator) + .concat(" Path to client key file") + .concat(lineSeparator) + .concat(" --max_tablets").concat(lineSeparator) + .concat(" Maximum number of tablets the client can poll for, default is 10") + .concat(lineSeparator); - CmdLineOpts configuration = new CmdLineOpts(); - configuration.initialize(commandLine); - return configuration; - } + public static CmdLineOpts createFromArgs(String[] args) throws Exception { + Options options = new Options(); - private String getRequiredOptionValue(CommandLine commandLine, String opt) throws Exception { - if (!commandLine.hasOption(opt)) { - throw new Exception (String.format("Command requires a %s argument", opt)); - } - return commandLine.getOptionValue(opt); - } + options.addOption("master_address", true, + "List of YB master ips to contact"); + options.addOption("table_name", true, + "Table to get change capture from in format .
"); + options.addOption("stream_id", true, + "Optional stream ID. Use this if you already have a CDC stream set" + + " up on the table"); - public void initialize(CommandLine commandLine) throws Exception { + // The config file can be passed from the command line using this option. + // A sample config.properties file has been provided under resources/config.properties + options.addOption("config_file", true, + "Path to config.properties file"); - if (commandLine.hasOption("master_addrs")) { - masterAddrs = commandLine.getOptionValue("master_addrs"); - } else { - LOG.info("Defaulting master_addrs to 127.0.0.1:7100"); - } + // Specify whether to create a new stream ID. + options.addOption("create_new_db_stream_id", false, + "Flag to create new cdc db stream id"); - String[] fullTableName = getRequiredOptionValue(commandLine, "table_name").split("\\."); + options.addOption("help", false, "Option to display help"); + options.addOption("show_config_help", false, + "Flag to display example for config.properties"); - if (fullTableName.length != 2) { - throw new Exception(String.format("Expected a namespace and table name for --table_name in" + - " format .
")); - } + options.addOption("disable_snapshot", false, + "Option to disable snapshot of a table"); + + options.addOption("ssl_cert_file", true, + "Option to provide the cert file if SSL/TLS enabled"); + + // Client certificate and key files. + options.addOption("client_cert_file", true, + "Option to provide client certificate file"); + options.addOption("client_key_file", true, + "Option to provide client key file"); + + // Maximum number of tablets the client can poll for, default is 10. + options.addOption("max_tablets", true, "Maximum number of tablets the client can " + + "poll for"); - namespaceName = fullTableName[0]; - tableName = fullTableName[1]; + // The interval at which the changes should be poplled at. + options.addOption("polling_interval", true, + "Interval at which the changes should be polled at"); - if (commandLine.hasOption("stream_id")) { - streamId = commandLine.getOptionValue("stream_id"); + // Do the actual arg parsing. + CommandLineParser parser = new BasicParser(); + CommandLine commandLine = null; + + try { + commandLine = parser.parse(options, args); + } catch (ParseException e) { + LOG.error("Error in args, use the --help option to see usage. Exception:", e); + System.exit(0); + } + + CmdLineOpts configuration = new CmdLineOpts(); + configuration.initialize(commandLine); + return configuration; + } + + private String getRequiredOptionValue(CommandLine commandLine, String opt) throws Exception { + if (!commandLine.hasOption(opt)) { + throw new Exception(String.format("Command requires a %s argument", opt)); + } + return commandLine.getOptionValue(opt); } - } + public void initialize(CommandLine commandLine) throws Exception { + // If the help options are there (--help, --show_config_help) then the relevant help + // message will be displayed and the program will exit from there. + if (commandLine.hasOption("help")) { + LOG.info(helpMessage); + System.exit(0); + } + if (commandLine.hasOption("show_config_help")) { + LOG.info(configFileHelpMessage); + System.exit(0); + } + + if (commandLine.hasOption("master_address")) { + masterAddrs = commandLine.getOptionValue("master_address"); + } else { + LOG.info("Looking out for the master_address in the cdc config file"); + } + + if (commandLine.hasOption("table_name")) { + LOG.info("Setting up table name from command line"); + + String[] fullTableName = commandLine.getOptionValue("table_name").split("\\."); + + if (fullTableName.length < 2) { + throw new Exception("Expected a namespace and table name for --table_name in the " + + "format .
\nRun with --help for more options"); + } else if (fullTableName.length > 3) { + throw new Exception("Table name exceeds the expected length, run with --help " + + "to see options"); + } + + if (fullTableName.length == 2) { + // This means that no schema name is provided and the format is .
+ // The first element in this case would be namespace name and second would be table name. + + namespaceName = fullTableName[0]; + tableName = fullTableName[1]; + } else if (fullTableName.length == 3) { + // This means schema name is provided in format ..
+ // The first element is namespace name, second is schema name and third is table name. + + namespaceName = fullTableName[0]; + tableName = fullTableName[1] + "." + fullTableName[2]; // Parsing would be done later. + } + } else { + LOG.info("Looking out for the table name in the cdc config file"); + } + + if (commandLine.hasOption("stream_id")) { + streamId = commandLine.getOptionValue("stream_id"); + } + + if (commandLine.hasOption("disable_snapshot")) { + enableSnapshot = false; + } else { + enableSnapshot = true; + } + + if (commandLine.hasOption("ssl_cert_file")) { + sslCertFile = commandLine.getOptionValue("ssl_cert_file"); + } + + if (commandLine.hasOption("client_cert_file")) { + clientCertFile = commandLine.getOptionValue("client_cert_file"); + } + + if (commandLine.hasOption("client_key_file")) { + clientKeyFile = commandLine.getOptionValue("client_key_file"); + } + + if (commandLine.hasOption("max_tablets")) { + maxTablets = Integer.parseInt(commandLine.getOptionValue("max_tablets")); + } + + if (commandLine.hasOption("polling_interval")) { + pollingInterval = Integer.parseInt(commandLine.getOptionValue("polling_interval")); + } + + // Check if a config file has been provided. + if (commandLine.hasOption("config_file")) { + LOG.info("Setting up config file path from command line"); + configFile = commandLine.getOptionValue("config_file"); + + PropertiesConfiguration propConfig = new PropertiesConfiguration(configFile); + if (commandLine.hasOption("table_name")) { + propConfig.setProperty("schema.name", namespaceName); + propConfig.setProperty("table.name", tableName); + LOG.info("Setting up table.name and schema.name from command line"); + } + if (commandLine.hasOption("stream_id") && commandLine + .hasOption("create_new_db_stream_id")) { + // This is not allowed as it would result in conflicting behaviour. + LOG.error("Providing both stream_id and create_new_db_stream_id" + + " together is not allowed"); + System.exit(0); + } + if (commandLine.hasOption("stream_id")) { + propConfig.setProperty("stream.id", streamId); + LOG.info("Setting up stream.id from command line"); + } + if (commandLine.hasOption("create_new_db_stream_id")) { + /* If this option is passed via command line then a new db stream id + * would be created even when the config.properties file contains a stream id + * the default behaviour in case this option is not passed would be to take whatever + * there is in the config file, if the stream id is empty then a new one would be + * created and if there is a pre-existing stream id in the file, it would be used. + */ + propConfig.setProperty("stream.id", ""); + } + if (commandLine.hasOption("master_address")) { + propConfig.setProperty("master.address", masterAddrs); + LOG.info("Setting up master.addrs from command line"); + } + + propConfig.setProperty("format", "proto"); + + propConfig.save(); + } else { + LOG.error("config_file not specified, use --config_file " + + " to specify a config file"); + + LOG.info(configFileHelpMessage); + System.exit(0); + } + } } diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentLogConnector.java b/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentLogConnector.java new file mode 100644 index 000000000000..d12f10007fbc --- /dev/null +++ b/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentLogConnector.java @@ -0,0 +1,200 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import com.google.common.net.HostAndPort; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.log4j.Logger; +import org.yb.client.*; +import org.yb.master.MasterDdlOuterClass; +import org.yb.util.ServerInfo; + +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +public class ConcurrentLogConnector { + private static final Logger LOG = Logger.getLogger(ConcurrentLogConnector.class); + private static AsyncYBClient client; + private static YBClient syncClient; + private static String CDC_CONFIG_FILE = ""; + private String format; + private final String PUBLIC_SCHEMA_NAME = "public"; + + private final ExecutorService executor; + private YBTable table; + List hps = new ArrayList<>(); + private final OutputClient outputClient; + private String streamId; + private boolean enableSnapshot; + private String sslCertFile; + private String clientCertFile; + private String clientKeyFile; + + private Set tableIds; + private String tableId = null; + private String namespace; + private String tableName; + + private Properties prop = new Properties(); + int concurrency = 1; + + private boolean stopExecution = false; + private int pollingInterval; + + public ConcurrentLogConnector(CmdLineOpts opts, OutputClient opClient) throws Exception { + InputStream input = new FileInputStream(opts.configFile); + CDC_CONFIG_FILE = opts.configFile; + enableSnapshot = opts.enableSnapshot; + + sslCertFile = opts.sslCertFile; + clientCertFile = opts.clientCertFile; + clientKeyFile = opts.clientKeyFile; + + pollingInterval = opts.pollingInterval; + + // Load a properties file. + prop.load(input); + format = prop.getProperty("format"); + namespace = prop.getProperty("schema.name"); + tableName = prop.getProperty("table.name"); + String schemaName = PUBLIC_SCHEMA_NAME; + + if (tableName.contains(".")) { + String[] splitVal = tableName.split("\\."); + schemaName = splitVal[0]; + tableName = splitVal[1]; + } + + LOG.info("Schema name while setting up: " + schemaName); + + LOG.info(String.format("Creating new YB client with master address %s", + prop.getProperty("master.address"))); + + client = new AsyncYBClient.AsyncYBClientBuilder(prop.getProperty("master.address")) + .sslCertFile(sslCertFile) + .sslClientCertFiles(clientCertFile, clientKeyFile) + .numTablets(opts.maxTablets) + .defaultAdminOperationTimeoutMs(Integer.parseInt( + prop.getProperty("admin.operation.timeout.ms"))) + .defaultOperationTimeoutMs(Integer.parseInt( + prop.getProperty("operation.timeout.ms"))) + .defaultSocketReadTimeoutMs(Integer.parseInt( + prop.getProperty("socket.read.timeout.ms"))) + .build(); + + syncClient = new YBClient(client); + concurrency = Integer.parseInt(prop.getProperty("num.io.threads")); + executor = Executors.newFixedThreadPool(concurrency, + new ThreadFactoryBuilder().setNameFormat("connector-%d").build()); + + ListTablesResponse tablesResp = syncClient.getTablesList(); + + for (MasterDdlOuterClass.ListTablesResponsePB.TableInfo tableInfo : tablesResp + .getTableInfoList()) { + if (tableInfo.getName().equals(tableName) && + tableInfo.getNamespace().getName().equals(namespace) && + tableInfo.getPgschemaName().equals(schemaName)) { + tableId = tableInfo.getId().toStringUtf8(); + // If the tableId is found, there's no point in iterating further. + break; + } + } + + // If tableId is not found, it's likely that it's not present, we should not proceed + // further in that case. + if (tableId == null) { + LOG.error(String.format("Could not find table with name %s.%s.%s", + namespace, schemaName, tableName)); + System.exit(0); + } + + tableIds = new HashSet<>(); + tableIds.add(tableId); + table = syncClient.openTableByUUID(tableId); + ListTabletServersResponse serversResp = syncClient.listTabletServers(); + for (ServerInfo serverInfo : serversResp.getTabletServersList()) { + hps.add(HostAndPort.fromParts(serverInfo.getHost(), serverInfo.getPort())); + } + outputClient = opClient; + streamId = prop.getProperty("stream.id"); // Getting this from passed options (opts). + input.close(); + } + + public void run() throws Exception { + if (streamId.isEmpty()) { + streamId = syncClient.createCDCStream(table, namespace, format, "IMPLICIT").getStreamId(); + LOG.debug(String.format("Created a new DB stream id: %s", streamId)); + + prop.setProperty("stream.id", streamId); + prop.store(new FileOutputStream(CDC_CONFIG_FILE), "null"); + } + LOG.info(String.format("DB stream id is %s", streamId)); + + List tabletLocations = table.getTabletsLocations(30000); + List>> tableIdsToTabletIdsMapList = new ArrayList<>(concurrency); + + for (int i = 0; i < concurrency; i++) { + tableIdsToTabletIdsMapList.add(new HashMap<>()); + } + int i = 0; + for (String tableId : tableIds) { + for (LocatedTablet tablet : tabletLocations) { + i++; + String tabletId = new String(tablet.getTabletId()); + tableIdsToTabletIdsMapList.get(i % concurrency).putIfAbsent(tableId, + new ArrayList<>()); + tableIdsToTabletIdsMapList.get(i % concurrency).get(tableId).add(tabletId); + } + } + + List runnables = tableIdsToTabletIdsMapList.stream().map( + tableIdsToTabletIds -> { + try { + return new ConcurrentPoller(syncClient, client, outputClient, streamId, + tableIdsToTabletIds, 2, format, stopExecution, + enableSnapshot); + } catch (IOException e) { + e.printStackTrace(); + } + return null; + }).filter(poller -> poller != null).map(poller -> (Runnable) () -> { + try { + while (true) { + poller.poll(); + Thread.sleep(pollingInterval); + } + } catch (Exception e) { + e.printStackTrace(); + } + }).collect(Collectors.toList()); + + List futures = runnables.stream() + .map(r -> executor.submit(r)).collect(Collectors.toList()); + + for (Future future : futures) { + future.get(); + } + } + + public void close() { + stopExecution = true; + } +} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentPoller.java b/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentPoller.java new file mode 100644 index 000000000000..d43bf62bb3b5 --- /dev/null +++ b/java/yb-cdc/src/main/java/org/yb/cdc/ConcurrentPoller.java @@ -0,0 +1,249 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import com.stumbleupon.async.Callback; +import com.stumbleupon.async.Deferred; +import org.apache.log4j.Logger; +import org.yb.cdc.util.Checkpoint; +import org.yb.client.*; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +public class ConcurrentPoller { + private static final Logger LOG = Logger.getLogger(ConcurrentPoller.class); + private final String streamId; + private final AsyncYBClient asyncYBClient; + private final OutputClient outputClient; + private final Semaphore requestBarrier; + private final String format; + private boolean stopExecution; + private boolean enableSnapshot; + + static final AbstractMap.SimpleImmutableEntry END_PAIR = + new AbstractMap.SimpleImmutableEntry("", ""); + + int concurrency; + + List> listTabletIdTableIdPair; + Map checkPointMap; + Map tableIdToTable; + AtomicInteger requestNum; + BlockingQueue> queue; + List> deferredList; + + YBClient synClient; + + public ConcurrentPoller(YBClient synClient, + AsyncYBClient client, + OutputClient outputClient, + String streamId, + Map> tableIdsToTabletIds, + int concurrency, + String format, + boolean stopExecution, + boolean enableSnapshot) throws IOException { + this.synClient = synClient; + this.asyncYBClient = client; + this.streamId = streamId; + this.format = format; + checkPointMap = new ConcurrentHashMap<>(); + tableIdToTable = new ConcurrentHashMap<>(); + requestBarrier = new Semaphore(concurrency); + requestNum = new AtomicInteger(); + this.outputClient = outputClient; + this.concurrency = concurrency; + deferredList = new ArrayList<>(); + this.stopExecution = stopExecution; + this.enableSnapshot = enableSnapshot; + + tableIdsToTabletIds.keySet().forEach(tabletId -> { + try { + tableIdToTable.put(tabletId, synClient.openTableByUUID(tabletId)); + } catch (Exception e) { + e.printStackTrace(); + } + }); + + listTabletIdTableIdPair = tableIdsToTabletIds.entrySet().stream() + .flatMap(e -> e.getValue().stream() + .map(v -> new AbstractMap.SimpleImmutableEntry<>(v, e.getKey()))) + .collect(Collectors.toList()); + queue = new LinkedBlockingQueue(); + initOffset(); + } + + private void initOffset() { + long term = 0; + long index = 0; + int writeId = 0; + + if (enableSnapshot) { + term = -1; + index = -1; + writeId = -1; + } + + long finalTerm = term; + long finalIndex = index; + int finalWriteId = writeId; + listTabletIdTableIdPair.forEach(entry -> + checkPointMap.put(entry.getKey(), new Checkpoint(finalTerm, finalIndex, + "".getBytes(), finalWriteId, 0))); + } + + public void poll() throws Exception { + final List result = new ArrayList(); + queue.addAll(listTabletIdTableIdPair); + queue.add(END_PAIR); + while (true) { + if (stopExecution) { + // This signals the CDCConsoleSubscriber to stop polling further and exit. + LOG.info("Signal received to close the CDCConsoleSubscriber, exiting..."); + System.exit(0); + } + + requestBarrier.acquireUninterruptibly(); + final AbstractMap.SimpleImmutableEntry entry = queue.take(); + if (entry.equals(END_PAIR)) { + requestBarrier.release(); + break; + } + final Checkpoint cp = checkPointMap.get(entry.getKey()); + final YBTable table = tableIdToTable.get(entry.getValue()); + + LOG.debug("Polling table: " + table + " tablet: " + entry.getKey() + + " with checkpoint " + cp); + Callback resCallback = new HandleResponse(table, entry.getKey(), result, requestBarrier); + Callback errCallback = new HandleFailure(requestBarrier); + + Deferred response = asyncYBClient.getChangesCDCSDK( + table, streamId, entry.getKey() /*tabletId*/, + cp.getTerm(), cp.getIndex(), cp.getKey(), cp.getWriteId(), cp.getSnapshotTime()); + + response.addCallback(resCallback); + response.addErrback(errCallback); + + deferredList.add(response); + } + + AtomicInteger totalException = new AtomicInteger(); + deferredList.forEach(getChangesResponseDeferred -> { + try { + getChangesResponseDeferred.join(120000); + } catch (Exception e) { + totalException.getAndIncrement(); + e.printStackTrace(); + } + }); + + if (listTabletIdTableIdPair.size() > 0) { + if (totalException.get() == listTabletIdTableIdPair.size()) { + LOG.error("Unable to poll further, all the nodes returned error"); + System.exit(1); + } else { + + } + } + } + + final class HandleFailure implements Callback { + private final Semaphore barrier; + + HandleFailure(Semaphore barrier) { + this.barrier = barrier; + } + + @Override + public Void call(Exception e) throws Exception { + barrier.release(); + LOG.debug("Releasing the requestbarrier" + barrier.availablePermits()); + + e.printStackTrace(); + if (e instanceof CDCErrorException) { + LOG.error("The error code is " + + ((CDCErrorException) e).getCDCError().getCode().getNumber()); + } + if (e instanceof CDCErrorException && + ((CDCErrorException) e).getCDCError().getCode().getNumber() == 25) { + LOG.error("Got the GetChangesResponse errback, error code is "); + } + + // TODO: Check all the exception here and throw only non-retryable exception as + // This is last chance for the application to retry. + throw e; + } + } + + final class HandleResponse implements Callback { + private final List result; + private final YBTable table; + private final String tabletId; + private final Semaphore barrier; + + HandleResponse(YBTable table, String tabletId, + List result, + Semaphore barrier) { + this.result = result; + this.table = table; + this.tabletId = tabletId; + this.barrier = barrier; + } + + public Void call(final GetChangesResponse response) { + return callPROTO(response); + } + + public Void callPROTO(final GetChangesResponse response) { + boolean noError = true; + + for (CdcService.CDCSDKProtoRecordPB record : response + .getResp() + .getCdcSdkProtoRecordsList()) { + try { + outputClient.applyChange(table, record); + result.add(record); + } catch (Exception e) { + e.printStackTrace(); + noError = false; + break; + } + } + + Checkpoint cp = null; + if (noError) { + cp = new Checkpoint( + response.getTerm(), + response.getIndex(), + response.getKey(), + response.getWriteId(), + response.getSnapshotTime()); + + checkPointMap.put(tabletId, cp); + LOG.debug("For tablet " + this.tabletId + " got the checkpoint " + cp); + } + + barrier.release(); + return null; + } + + public String toString() { + return "Handle Response"; + } + } +} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/LogClient.java b/java/yb-cdc/src/main/java/org/yb/cdc/LogClient.java index bdafda307332..96178fc9cb2b 100644 --- a/java/yb-cdc/src/main/java/org/yb/cdc/LogClient.java +++ b/java/yb-cdc/src/main/java/org/yb/cdc/LogClient.java @@ -17,10 +17,32 @@ import org.yb.client.YBTable; public class LogClient implements OutputClient { + long inserts = 0; + long updates = 0; + long deletes = 0; + long writes = 0; + long snapshotRecords = 0; + private static final Logger LOG = Logger.getLogger(LogClient.class); @Override - public void applyChange(YBTable table, CdcService.CDCRecordPB changeRecord) throws Exception { + public void applyChange(YBTable table, CdcService.CDCSDKProtoRecordPB changeRecord) { LOG.info(changeRecord.toString()); + switch (changeRecord.getRowMessage().getOp()) { + case INSERT: + ++inserts; + break; + case UPDATE: + ++updates; + break; + case DELETE: + ++deletes; + break; + case READ: + ++snapshotRecords; + break; + } + LOG.info(String.format("Inserts: %d, Updates: %d, Deletes: %d, Snapshot Records: %d", + inserts, updates, deletes, snapshotRecords)); } } diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/LogConnector.java b/java/yb-cdc/src/main/java/org/yb/cdc/LogConnector.java deleted file mode 100644 index a719273d3c45..000000000000 --- a/java/yb-cdc/src/main/java/org/yb/cdc/LogConnector.java +++ /dev/null @@ -1,112 +0,0 @@ -// Copyright (c) YugaByte, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except -// in compliance with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software distributed under the License -// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express -// or implied. See the License for the specific language governing permissions and limitations -// under the License. -// - -package org.yb.cdc; - -import com.google.common.net.HostAndPort; -import org.yb.client.YBTable; -import org.yb.client.AsyncYBClient; -import org.yb.client.YBClient; -import org.yb.client.ListTablesResponse; -import org.yb.client.ListTabletServersResponse; -import org.yb.client.LocatedTablet; - -import java.util.ArrayList; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.*; -import org.apache.log4j.Logger; -import org.yb.master.MasterDdlOuterClass; -import org.yb.util.ServerInfo; - -class LogConnector { - private static final Logger LOG = Logger.getLogger(LogConnector.class); - - private static AsyncYBClient client; - private static YBClient syncClient; - - private ExecutorService executor; - - private static final int DEFAULT_TIMEOUT = 30000; - private static final int DEFAULT_NUM_THREADS = 8; - - private YBTable table; - - private List pollers = new ArrayList<>(); - - List hps = new ArrayList<>(); - - private OutputClient outputClient; - - private String streamId; - - public LogConnector(CmdLineOpts opts) throws Exception { - LOG.info("Creating new YB client..."); - client = new AsyncYBClient.AsyncYBClientBuilder(opts.masterAddrs) - .defaultAdminOperationTimeoutMs(DEFAULT_TIMEOUT) - .defaultOperationTimeoutMs(DEFAULT_TIMEOUT) - .defaultSocketReadTimeoutMs(DEFAULT_TIMEOUT) - .build(); - - syncClient = new YBClient(client); - - executor = Executors.newFixedThreadPool(DEFAULT_NUM_THREADS); - - String tableId = null; - ListTablesResponse tablesResp = syncClient.getTablesList(); - for (MasterDdlOuterClass.ListTablesResponsePB.TableInfo tableInfo : - tablesResp.getTableInfoList()) { - if (tableInfo.getName().equals(opts.tableName) && - tableInfo.getNamespace().getName().equals(opts.namespaceName)) { - tableId = tableInfo.getId().toStringUtf8(); - } - } - - if (tableId == null) { - LOG.error(String.format("Could not find a table with name %s.%s", - opts.namespaceName, opts.tableName)); - System.exit(0); - } - - table = syncClient.openTableByUUID(tableId); - - ListTabletServersResponse serversResp = syncClient.listTabletServers(); - for (ServerInfo serverInfo : serversResp.getTabletServersList()) { - hps.add(HostAndPort.fromParts(serverInfo.getHost(), serverInfo.getPort())); - } - - outputClient = new LogClient(); - - streamId = opts.streamId; - } - - public void run() throws Exception { - Random rand = new Random(); - - HostAndPort hp = hps.get(rand.nextInt(hps.size())); - if (streamId.isEmpty()) { - streamId = syncClient.createCDCStream(hp, table.getTableId()).getStreamId(); - LOG.info(String.format("Created new stream with id %s", streamId)); - } - - List tabletLocations = table.getTabletsLocations(DEFAULT_TIMEOUT); - - for (LocatedTablet tablet : tabletLocations) { - String tabletId = new String(tablet.getTabletId()); - LOG.info(String.format("Polling for new tablet %s", tabletId)); - Poller poller = new Poller(outputClient, hps, table, streamId, tabletId, executor, client); - pollers.add(poller); - poller.poll(); - } - } -} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/Main.java b/java/yb-cdc/src/main/java/org/yb/cdc/Main.java deleted file mode 100644 index c85a2c2a674c..000000000000 --- a/java/yb-cdc/src/main/java/org/yb/cdc/Main.java +++ /dev/null @@ -1,49 +0,0 @@ -package org.yb.cdc; - -import org.apache.log4j.ConsoleAppender; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.PatternLayout; - -public class Main { - private static final Logger LOG = Logger.getLogger(Main.class); - - private LogConnector connector; - - public Main(CmdLineOpts cmdLineOpts) throws Exception { - connector = new LogConnector(cmdLineOpts); - } - - public void run() { - try { - connector.run(); - } catch (Exception e) { - LOG.error("Application ran into error: ", e); - System.exit(0); - } - } - - public static void main(String[] args) throws Exception { - LOG.info("Starting CDC Console Connector..."); - // First remove all appenders. - Logger.getLogger("org.yb.cdc").removeAppender("YBConsoleLogger"); - Logger.getRootLogger().removeAppender("YBConsoleLogger");; - - // Create the console appender. - ConsoleAppender console = new ConsoleAppender(); - console.setName("YBConsoleLogger"); - String PATTERN = "%d [%p|%c|%C{1}] %m%n"; - console.setLayout(new PatternLayout(PATTERN)); - console.setThreshold(Level.INFO); - console.activateOptions(); - - Logger.getLogger("org.yb.cdc").addAppender(console); - Logger.getLogger("org.yb.cdc").setAdditivity(false); - - CmdLineOpts configuration = CmdLineOpts.createFromArgs(args); - Main main = new Main(configuration); - main.run(); - } - - -} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/OutputClient.java b/java/yb-cdc/src/main/java/org/yb/cdc/OutputClient.java index b7279bff7a37..26d46999ab2f 100644 --- a/java/yb-cdc/src/main/java/org/yb/cdc/OutputClient.java +++ b/java/yb-cdc/src/main/java/org/yb/cdc/OutputClient.java @@ -16,5 +16,6 @@ import org.yb.client.YBTable; public interface OutputClient { - public void applyChange(YBTable table, CdcService.CDCRecordPB changeRecord) throws Exception; + public void applyChange(YBTable table, + CdcService.CDCSDKProtoRecordPB changeRecord) throws Exception; } diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/Poller.java b/java/yb-cdc/src/main/java/org/yb/cdc/Poller.java deleted file mode 100644 index fa89f1808a5b..000000000000 --- a/java/yb-cdc/src/main/java/org/yb/cdc/Poller.java +++ /dev/null @@ -1,111 +0,0 @@ -// Copyright (c) YugaByte, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except -// in compliance with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software distributed under the License -// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express -// or implied. See the License for the specific language governing permissions and limitations -// under the License. -// - -package org.yb.cdc; - -import com.stumbleupon.async.Callback; -import org.apache.log4j.Logger; -import org.yb.client.AsyncYBClient; -import org.yb.client.GetChangesResponse; -import org.yb.client.YBTable; - -import java.util.Random; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.List; - -import com.google.common.net.HostAndPort; - - -public class Poller { - private static final Logger LOG = Logger.getLogger(Poller.class); - - private long term; - private long index; - private String streamId; - private String tabletId; - private ExecutorService executor; - private YBTable table; - private AsyncYBClient client; - private List hps; - private OutputClient outputClient; - - public Poller(OutputClient outputClient, List hps, YBTable table, String streamId, - String tabletId, ExecutorService executor, AsyncYBClient client) { - this.streamId = streamId; - this.tabletId = tabletId; - this.term = 0; - this.index = 0; - this.executor = executor; - this.client = client; - this.table = table; - this.hps = hps; - this.outputClient = outputClient; - } - - public Void poll() { - executor.submit(new Callable() { - @Override - public Void call() throws Exception { - return doPoll(); - } - }); - return null; - } - - private Void doPoll() { - Random rand = new Random(); - HostAndPort hp = hps.get(rand.nextInt(hps.size())); - - client.getChanges(hp, table, streamId, tabletId, term, index, - new Callback() { - @Override - public Void call(GetChangesResponse getChangesResponse) throws Exception { - return handlePoll(getChangesResponse); - } - }); - return null; - } - - private Void handlePoll(GetChangesResponse getChangesResponse) { - executor.submit(new Callable() { - @Override - public Void call() throws Exception { - return doHandlePoll(getChangesResponse); - } - }); - return null; - } - - private Void doHandlePoll(GetChangesResponse getChangesResponse) throws Exception { - // Handle the poll - if (getChangesResponse.getResp().hasError()) { - if (getChangesResponse.getResp().getError().getCode() == - CdcService.CDCErrorPB.Code.INVALID_REQUEST) { - LOG.error("Invalid Request"); - System.exit(0); - } - return poll(); - } - - for (org.yb.cdc.CdcService.CDCRecordPB record : getChangesResponse.getResp().getRecordsList()) { - outputClient.applyChange(table, record); - } - - this.term = getChangesResponse.getResp().getCheckpoint().getOpId().getTerm(); - this.index = getChangesResponse.getResp().getCheckpoint().getOpId().getIndex(); - - return poll(); - } - -} diff --git a/java/yb-cdc/src/main/java/org/yb/cdc/util/Checkpoint.java b/java/yb-cdc/src/main/java/org/yb/cdc/util/Checkpoint.java new file mode 100644 index 000000000000..f2850e897a87 --- /dev/null +++ b/java/yb-cdc/src/main/java/org/yb/cdc/util/Checkpoint.java @@ -0,0 +1,78 @@ +package org.yb.cdc.util; + +import com.google.common.base.Objects; +import org.yb.client.GetChangesResponse; + +import java.util.Arrays; + +public class Checkpoint { + private long term; + private long index; + private byte[] key; + private int writeId; + private long snapshotTime; + + public Checkpoint(long term, long index, byte[] key, int writeId, long snapshotTime) { + this.term = term; + this.index = index; + this.key = key; + this.writeId = writeId; + this.snapshotTime = snapshotTime; + } + + public static Checkpoint from(GetChangesResponse resp) { + return new Checkpoint(resp.getTerm(), resp.getIndex(), resp.getKey(), + resp.getWriteId(), resp.getSnapshotTime()); + } + + @Override + public String toString() { + return "Checkpoint{" + + "term=" + term + + ", index=" + index + + ", key=" + Arrays.toString(key) + + ", writeId=" + writeId + + ", snapshotTime=" + snapshotTime + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + Checkpoint that = (Checkpoint) o; + return term == that.getTerm() && index == that.getIndex() && writeId == that.getWriteId() + && java.util.Objects.equals(key, that.getKey()); + } + + @Override + public int hashCode() { + return Objects.hashCode(term, index, key, writeId); + } + + public long getTerm() { + return term; + } + + public long getIndex() { + return index; + } + + public byte[] getKey() { + return key; + } + + public int getWriteId() { + return writeId; + } + + public long getSnapshotTime() { + return snapshotTime; + } +} diff --git a/java/yb-cdc/src/main/resources/config.properties b/java/yb-cdc/src/main/resources/config.properties new file mode 100644 index 000000000000..a9c22069b06a --- /dev/null +++ b/java/yb-cdc/src/main/resources/config.properties @@ -0,0 +1,11 @@ +num.io.threads=1 +master.address=127.0.0.1\:7100 +schema.name=yugabyte +socket.read.timeout.ms=60000 +operation.timeout.ms=60000 +admin.operation.timeout.ms=60000 +master.address=127.0.0.1:7100 +table.name=test +format=PROTO +stream.id= +tablets.list= diff --git a/java/yb-cdc/src/main/resources/log4j.properties b/java/yb-cdc/src/main/resources/log4j.properties new file mode 100644 index 000000000000..dd44baee6268 --- /dev/null +++ b/java/yb-cdc/src/main/resources/log4j.properties @@ -0,0 +1,17 @@ +# Configure logging for testing: optionally with log file +#log4j.rootLogger=INFO, stdout +log4j.rootLogger=INFO, stdout, fileAppender + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d %p [%c] (%t) - %m%n + +log4j.appender.fileAppender=org.apache.log4j.RollingFileAppender +log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.fileAppender.layout.ConversionPattern=%d %p [%c] (%t) - %m%n + +# log4j.appender.fileAppender.layout.ConversionPattern=%d (%t) [%p - %l] %m%n +# [%t] %-5p %c %x - %m%n +log4j.appender.fileAppender.File=cdcsubscriber.log + +log4j.logger.org.yb.client = INFO diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCStream.java b/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCStream.java new file mode 100644 index 000000000000..5f2843b96cb0 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCStream.java @@ -0,0 +1,237 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.client.*; +import org.yb.util.YBTestRunnerNonTsanOnly; + + +import java.util.ArrayList; +import java.util.List; + +import static org.yb.AssertionWrappers.*; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestCDCStream extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestCDCStream.class); + + private final String DEFAULT_NAMESPACE = "yugabyte"; + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + } + + /** + * Test to verify creation of a CDC stream when correct parameters are provided + */ + @Test + public void testStreamCreationWithCorrectParams() { + try { + statement.execute("create table test (a int primary key, b int, c numeric);"); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + YBClient myClient = testSubscriber.getSyncClient(); + assertNotNull(myClient); + + // The below table would only be for the table yugabyte.test + YBTable table = testSubscriber.getTable(); + assertNotNull(table); + + CreateCDCStreamResponse resp = + myClient.createCDCStream(table, DEFAULT_NAMESPACE, "proto", "implicit"); + + assertNotNull(resp); + assertFalse(resp.getStreamId().isEmpty()); + } catch (Exception e) { + LOG.error("Test to verify correct stream creation failed", e); + fail(); + } + } + + /** + * Negative test: Trying to create a CDC stream on a table which does not exist. It would throw + * an exception saying "Table with identifier not found: OBJECT_NOT_FOUND" + */ + @Test + public void testStreamCreationOnNonExistingTable() { + try { + // The table "test" does not exist in this test and we won't create one too. + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + + boolean masterExceptionThrown = false; + // Try creating a stream. + // The following function would look up for a table yugabyte.test by default + // and try to create stream on that, it would fail eventually. + try { + testSubscriber.createStream("proto"); + } catch (MasterErrorException me) { + // MasterErrorException would be thrown with message: + // Table with identifier not found: OBJECT_NOT_FOUND + masterExceptionThrown = true; + } + + assertTrue(masterExceptionThrown); + } catch (Exception e) { + LOG.error("Test to verify failure on creating stream on a non-existing table failed", e); + fail(); + } + } + + /** + * Negative test: Provided a wrong namespace and trying to look for the table there and then + * create the CDC Stream on that table. This would throw an exception while creating stream + */ + @Test + public void testCreateStreamWithInvalidNamespace() { + try { + statement.execute("create table test (a int primary key, b int, c numeric);"); + + // Dummy insert. + int res = statement.executeUpdate("insert into test values (1, 2, 20.34);"); + assertEquals(1, res); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + YBClient myClient = testSubscriber.getSyncClient(); + assertNotNull(myClient); + + // The below table would only be for the table yugabyte.test + YBTable table = testSubscriber.getTable(); + assertNotNull(table); + + boolean exceptionThrown = false; + try { + CreateCDCStreamResponse resp = + myClient.createCDCStream(table, "non_existing_namespace", "proto", "implicit"); + } catch (Exception e) { + // The above try block would throw an exception since we are trying to create a stream + // on a namespace which doesn't exist. + assertTrue(e.getMessage().contains("Keyspace name not found: non_existing_namespace")); + exceptionThrown = true; + } + assertTrue(exceptionThrown); + } catch (Exception e) { + LOG.error("Test to verify failure with invalid namespace failed with exception", e); + fail(); + } + } + + /** + * Test to verify that creating a stream on a table with no primary key fails. While trying to + * create the CDC stream, an exception would be thrown while creating stream. + */ + @Test + public void testStreamCreationWithoutPrimaryKey() { + try { + statement.execute("create table test (a int, b int);"); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + YBClient myClient = testSubscriber.getSyncClient(); + assertNotNull(myClient); + + // The below table would only be for the table yugabyte.test + YBTable table = testSubscriber.getTable(); + assertNotNull(table); + + // After the fix in GitHub Issue [#10945], no exception would be thrown if we try to create + // a stream on a database if it contains a table with no primary key. + boolean exceptionThrown = false; + try { + CreateCDCStreamResponse resp = + myClient.createCDCStream(table, DEFAULT_NAMESPACE, "proto", "implicit"); + } catch (Exception e) { + // The try block would throw an exception since we are trying to create stream on + // a table with no primary key. + exceptionThrown = true; + } + assertFalse(exceptionThrown); + + } catch (Exception e) { + LOG.error("Test to verify attaching cdc on a table with no primary key " + + "failed with exception", e); + fail(); + } + } + + @Test + public void testPollingWithWrongStreamId() { + try { + assertFalse(statement.execute("create table test (a int primary key, b int);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + String someRandomStreamId = "75fb51d1va000ib0h88a710v140a4f51"; + + // Explicitly changing the automatically generated db stream id. + testSubscriber.setDbStreamId(someRandomStreamId); + + int dummyInsert = statement.executeUpdate("insert into test values (1, 2);"); + assertEquals(1, dummyInsert); + + List outputList = new ArrayList<>(); + boolean cdcExceptionCaught = false; + try { + // CDCErrorException would be thrown since the DB stream Id doesn't exist. + testSubscriber.getResponseFromCDC(outputList); + } catch (CDCErrorException ce) { + assertTrue(ce.getMessage().contains("75fb51d1va000ib0h88a710v140a4f51")); + // NOT_FOUND[code 1]: Could not find CDC stream + cdcExceptionCaught = true; + } + assertEquals(0, outputList.size()); + assertTrue(cdcExceptionCaught); + } catch (Exception e) { + LOG.error("Test to verify failure on polling with wrong DB stream id failed", e); + fail(); + } + } + + @Test + public void testStreamCreationUsingYbAdmin() { + try { + String dbStreamId = createDBStreamUsingYbAdmin(getMasterAddresses(), "yugabyte"); + + assertNotNull(dbStreamId); + assertFalse(dbStreamId.isEmpty()); + } catch (Exception e) { + LOG.error("Test to verify stream creation using yb-admin failed", e); + fail(); + } + } + + @Test + public void testStreamCreationAndDeletionYbAdmin() { + try { + String dbStreamId = createDBStreamUsingYbAdmin(getMasterAddresses(), "yugabyte"); + + assertNotNull(dbStreamId); + assertFalse(dbStreamId.isEmpty()); + + String deletedStreamId = deleteDBStreamUsingYbAdmin(getMasterAddresses(), dbStreamId); + assertEquals(dbStreamId, deletedStreamId); + } catch (Exception e) { + LOG.error("Test to verify stream creation and deletion via yb-admin failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCSubscriber.java b/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCSubscriber.java new file mode 100644 index 000000000000..145ef81aaa9e --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/TestCDCSubscriber.java @@ -0,0 +1,57 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import static org.yb.AssertionWrappers.*; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestCDCSubscriber extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestCDCSubscriber.class); + + @Before + public void setUp() throws Exception { + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c int);"); + } + + @Test + public void testSettingInvalidFormat() throws Exception { + LOG.info("Starting testSettingInvalidStreamFormat"); + + boolean runtimeExceptionThrown = false; + try { + // Dummy insert statement. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + // This will throw a RuntimeException currently, but on the server side, if anything other + // than proto is specified then it would be defaulted to json. + testSubscriber.createStream("some.invalid.format"); + } catch (RuntimeException re) { + runtimeExceptionThrown = true; + } + + assertTrue(runtimeExceptionThrown); + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/TestCheckpoint.java b/java/yb-cdc/src/test/java/org/yb/cdc/TestCheckpoint.java new file mode 100644 index 000000000000..cd16ef321543 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/TestCheckpoint.java @@ -0,0 +1,169 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.Logger; +import org.junit.runner.RunWith; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.client.GetCheckpointResponse; +import org.yb.client.SetCheckpointResponse; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Test; +import org.yb.util.YBTestRunnerNonTsanOnly; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestCheckpoint extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestCheckpoint.class); + + @Before + public void setUp() throws Exception { + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c int);"); + } + + /** + * Testing the getCheckpoint API. + * @throws Exception + */ + @Test + public void testGetCheckpointResponse() throws Exception { + LOG.info("Starting testGetCheckpointResponse"); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + // Dummy insert statement. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + GetCheckpointResponse resp = testSubscriber.getCheckpoint(); + + assertNotNull(resp); + } + + /** + * Test to verify that we can set a checkpoint when the stream is set in EXPLICIT mode + */ + @Test + public void testCheckpointing() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto", "EXPLICIT"); // setting a stream with PROTO format + + // Dummy insert statement. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + GetCheckpointResponse resp = testSubscriber.getCheckpoint(); + if(resp == null) { + LOG.error("Null response received as GetCheckpointResponse"); + fail(); + } + + long cpTerm = 2; + long cpIndex = 9; + SetCheckpointResponse setResp = testSubscriber.setCheckpoint(cpTerm, cpIndex); + if (setResp == null) { + LOG.error("Cannot set checkpoint, null response received as SetCheckpointResponse"); + } + + resp = testSubscriber.getCheckpoint(); + + assertEquals(cpTerm, resp.getTerm()); + assertEquals(cpIndex, resp.getIndex()); + } catch (Exception e) { + LOG.error("Test to verify checkpointing failed with exception", e); + fail(); + } + } + + /** + * Test to verify that if in EXPLICIT mode, when we try to set the checkpoint with a negative + * index, it won't do anything on the server side and the checkpoint would remain unchanged + */ + @Test + public void testSettingNegativeIndexAsCheckpoint() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto", "EXPLICIT"); // setting a stream with PROTO format + + // Dummy insert statement. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + int rowsAffected2 = statement.executeUpdate("insert into test values (11, 22, 33);"); + assertEquals(1, rowsAffected2); + + GetCheckpointResponse respBeforeSetting = testSubscriber.getCheckpoint(); + if(respBeforeSetting == null) { + LOG.error("Null response received as GetCheckpointResponse"); + fail(); + } + + SetCheckpointResponse setResp = testSubscriber.setCheckpoint(1, -3); + + GetCheckpointResponse respAfterSetting = testSubscriber.getCheckpoint(); + + // Now even if the checkpoint is set negatively, + // it would remain the same as the one before setting it explicitly + // since a negative index is illegal. + assertEquals(respBeforeSetting.getTerm(), respAfterSetting.getTerm()); + assertEquals(respBeforeSetting.getIndex(), respAfterSetting.getIndex()); + } catch (Exception e) { + LOG.error("Test to verify failure while setting checkpoint with negative index " + + "failed with exception", e); + fail(); + } + } + + /** + * Test to verify that trying to set a checkpoint in IMPLICIT mode fails + */ + @Test + public void testSettingCheckpointWithImplicit() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + // Setting the checkpoint type as IMPLICIT. + testSubscriber.createStream("proto", "IMPLICIT"); + + // Dummy insert. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + GetCheckpointResponse respBeforeSetting = testSubscriber.getCheckpoint(); + + long cpTerm = 1; + long cpIndex = 7; + + SetCheckpointResponse resp = testSubscriber.setCheckpoint(cpTerm, cpIndex); + + // Checkpoint will be set to the specified value, + // we are just checking if checkpoint can be set in IMPLICIT mode. + GetCheckpointResponse respAfterSetting = testSubscriber.getCheckpoint(); + + assertNotEquals(respBeforeSetting.getTerm(), respAfterSetting.getTerm()); + assertNotEquals(respBeforeSetting.getIndex(), respAfterSetting.getIndex()); + + assertEquals(cpTerm, respAfterSetting.getTerm()); + assertEquals(cpIndex, respAfterSetting.getIndex()); + } catch (Exception e) { + LOG.error("Test to verify behaviour when setting checkpoint in IMPLICIT mode failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/TestDBStreamInfo.java b/java/yb-cdc/src/test/java/org/yb/cdc/TestDBStreamInfo.java new file mode 100644 index 000000000000..56cafa9af282 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/TestDBStreamInfo.java @@ -0,0 +1,53 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.client.GetDBStreamInfoResponse; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import static org.yb.AssertionWrappers.*; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestDBStreamInfo extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestDBStreamInfo.class); + + @Before + public void setUp() throws Exception { + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c int);"); + } + + @Test + public void testDBStreamInfoResponse() throws Exception { + LOG.info("Starting testDBStreamInfoResponse"); + + // Inserting a dummy row. + int rowsAffected = statement.executeUpdate("insert into test values (1, 2, 3);"); + assertEquals(1, rowsAffected); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + GetDBStreamInfoResponse resp = testSubscriber.getDBStreamInfo(); + + assertNotEquals(0, resp.getTableInfoList().size()); + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/TestGetChanges.java b/java/yb-cdc/src/test/java/org/yb/cdc/TestGetChanges.java new file mode 100644 index 000000000000..9983ee8ab9e8 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/TestGetChanges.java @@ -0,0 +1,92 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.client.*; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.util.ArrayList; +import java.util.List; + +import static org.yb.AssertionWrappers.*; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestGetChanges extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestGetChanges.class); + + @Before + public void setUp() throws Exception { + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c numeric);"); + } + + @Ignore("this test passes if you would provide 127.0.0.1:7100 to CDCSubscriber()") + @Test + public void testGettingChangesWithNegativeIndex() { + try { + // This test passes if you would provide 127.0.0.1:7100 to CDCSubscriber() + // The behaviour is flaky, so the test is disabled. + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + YBClient myClient = testSubscriber.getSyncClient(); + assertNotNull(myClient); + + // The below table would only be for the table yugabyte.test + YBTable table = testSubscriber.getTable(); + assertNotNull(table); + + CreateCDCStreamResponse resp = + myClient.createCDCStream(table, "yugabyte", "proto", "explicit"); + + assertNotNull(resp); + + // Dummy insert. + int res = statement.executeUpdate("insert into test values (1, 2, 20.34);"); + assertEquals(1, res); + + String dbStreamId = resp.getStreamId(); + + List locatedTablets = table.getTabletsLocations(30000); + List tabletIds = new ArrayList<>(); + + for (LocatedTablet tablet : locatedTablets) { + String tabletId = new String(tablet.getTabletId()); + tabletIds.add(tabletId); + } + + boolean exceptionThrown = false; + for (String tabletId : tabletIds) { + // An exception would be thrown for an index less than 0. + try { + GetChangesResponse changesResponse = + myClient.getChangesCDCSDK(table, dbStreamId, tabletId, 0, -1, new byte[]{}, 0, 0L); + } catch (Exception e) { + exceptionThrown = true; + break; + } + } + assertTrue(exceptionThrown); + } catch (Exception e) { + LOG.error("Test to verify failure on requesting changes from a negative index failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCBaseClass.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCBaseClass.java new file mode 100644 index 000000000000..e94b2000c6b3 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCBaseClass.java @@ -0,0 +1,768 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import static com.google.common.base.Preconditions.*; +import static org.yb.AssertionWrappers.*; +import static org.yb.util.BuildTypeUtil.isASAN; +import static org.yb.util.BuildTypeUtil.isTSAN; + +import com.google.common.net.HostAndPort; + +import org.apache.commons.io.IOUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import com.yugabyte.jdbc.PgArray; +import com.yugabyte.util.PGobject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yb.client.IsInitDbDoneResponse; +import org.yb.client.TestUtils; +import org.yb.minicluster.*; +import org.yb.util.EnvAndSysPropertyUtil; +import org.yb.util.BuildTypeUtil; +import org.yb.util.YBBackupUtil; + +import java.io.File; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.sql.*; +import java.util.*; +import java.util.stream.Collectors; + +// The majority of the functions in this class has been duplicated from one of the base classes +// in yb-pgsql project. It seemed necessary in order to avoid adding a complete dependency +// on the same project. This duplication will be removed in future releases. +public class CDCBaseClass extends BaseMiniClusterTest { + private static final Logger LOG = LoggerFactory.getLogger(CDCBaseClass.class); + + protected String CDC_BATCH_SIZE_GFLAG = "cdc_snapshot_batch_size"; + protected String CDC_INTENT_SIZE_GFLAG = "cdc_max_stream_intent_records"; + + // Postgres settings. + protected static final String DEFAULT_PG_DATABASE = "yugabyte"; + protected static final String DEFAULT_PG_USER = "yugabyte"; + protected static final String DEFAULT_PG_PASS = "yugabyte"; + + // CQL and Redis settings, will be reset before each test via resetSettings method. + protected boolean startCqlProxy = false; + protected boolean startRedisProxy = false; + + protected static Connection connection; + protected Statement statement; + + protected File pgBinDir; + + protected static boolean pgInitialized = false; + + @Override + protected int getNumShardsPerTServer() { + return 1; + } + + @Override + protected int getReplicationFactor() { + return 1; + } + + protected String getMasterAddresses() { + return miniCluster.getMasterAddresses(); + } + + protected HostAndPort getTserverHostAndPort() { + // Since we are dealing with a single tserver, so it is safe to assume that the below line + // would return only one element, that too the one we need. + return miniCluster.getTabletServers().keySet().iterator().next(); + } + + protected Integer getYsqlPrefetchLimit() { + return null; + } + + protected Integer getYsqlRequestLimit() { + return null; + } + + /** empty helper function */ + protected void setUp() throws Exception { + } + + /** + * @return flags shared between tablet server and initdb + */ + @Override + protected Map getTServerFlags() { + Map flagMap = super.getTServerFlags(); + + if (isTSAN() || isASAN()) { + flagMap.put("pggate_rpc_timeout_secs", "120"); + } + flagMap.put("start_cql_proxy", String.valueOf(startCqlProxy)); + flagMap.put("start_redis_proxy", String.valueOf(startRedisProxy)); + + // Setup flag for postgres test on prefetch-limit when starting tserver. + if (getYsqlPrefetchLimit() != null) { + flagMap.put("ysql_prefetch_limit", getYsqlPrefetchLimit().toString()); + } + + if (getYsqlRequestLimit() != null) { + flagMap.put("ysql_request_limit", getYsqlRequestLimit().toString()); + } + + flagMap.put("ysql_beta_features", "true"); + flagMap.put("ysql_sleep_before_retry_on_txn_conflict", "false"); + flagMap.put("ysql_max_write_restart_attempts", "2"); + + return flagMap; + } + + @Override + protected Map getMasterFlags() { + Map flagMap = super.getMasterFlags(); + flagMap.put("client_read_write_timeout_ms", + String.valueOf(BuildTypeUtil.adjustTimeout(120000))); + flagMap.put("memory_limit_hard_bytes", String.valueOf(2L * 1024 * 1024 * 1024)); + return flagMap; + } + + @Override + protected void customizeMiniClusterBuilder(MiniYBClusterBuilder builder) { + super.customizeMiniClusterBuilder(builder); + builder.enableYsql(true); + } + + @Before + public void initYBBackupUtil() { + YBBackupUtil.setMasterAddresses(masterAddresses); + YBBackupUtil.setPostgresContactPoint(miniCluster.getPostgresContactPoints().get(0)); + } + + @Before + public void initPostgresBefore() throws Exception { + if (pgInitialized) + return; + + LOG.info("Loading PostgreSQL JDBC driver"); + Class.forName("com.yugabyte.Driver"); + + // Postgres bin directory. + pgBinDir = new File(TestUtils.getBuildRootDir(), "postgres/bin"); + + LOG.info("Waiting for initdb to complete on master"); + TestUtils.waitFor( + () -> { + IsInitDbDoneResponse initdbStatusResp = miniCluster.getClient().getIsInitDbDone(); + if (initdbStatusResp.hasError()) { + throw new RuntimeException( + "Could not request initdb status: " + initdbStatusResp.getServerError()); + } + String initdbError = initdbStatusResp.getInitDbError(); + if (initdbError != null && !initdbError.isEmpty()) { + throw new RuntimeException("initdb failed: " + initdbError); + } + return initdbStatusResp.isDone(); + }, + 600000); + LOG.info("initdb has completed successfully on master"); + + if (connection != null) { + LOG.info("Closing previous connection"); + connection.close(); + connection = null; + } + + connection = getConnectionBuilder().connect(); + pgInitialized = true; + } + + @Override + protected void resetSettings() { + super.resetSettings(); + startCqlProxy = false; + startRedisProxy = false; + } + + protected ConnectionBuilder getConnectionBuilder() { + return new ConnectionBuilder(miniCluster); + } + + @After + public void cleanUpAfter() throws Exception { + LOG.info("Cleaning up after {}", getCurrentTestMethodName()); + if (connection == null) { + LOG.warn("No connection created, skipping cleanup"); + return; + } + + // If root connection was closed, open a new one for cleaning. + if (connection.isClosed()) { + connection = getConnectionBuilder().connect(); + } + + try (Statement stmt = connection.createStatement()) { + stmt.execute("RESET SESSION AUTHORIZATION"); + stmt.execute("ROLLBACK"); + stmt.execute("DISCARD TEMP"); + } + + cleanUpCustomDatabases(); + + if (isClusterNeedsRecreation()) { + pgInitialized = false; + } + } + + /** + * Removes all databases excluding `postgres`, `yugabyte`, `system_platform`, `template1`, and + * `template2`. Any lower-priority cleaners should only clean objects in one of the remaining + * three databases, or cluster-wide objects (e.g. roles). + */ + private void cleanUpCustomDatabases() throws Exception { + LOG.info("Cleaning up custom databases"); + try (Statement stmt = connection.createStatement()) { + for (int i = 0; i < 2; i++) { + try { + List databases = getRowList(stmt, + "SELECT datname FROM pg_database" + + " WHERE datname <> 'template0'" + + " AND datname <> 'template1'" + + " AND datname <> 'postgres'" + + " AND datname <> 'yugabyte'" + + " AND datname <> 'system_platform'").stream().map(r -> r.getString(0)) + .collect(Collectors.toList()); + + for (String database : databases) { + LOG.info("Dropping database '{}'", database); + stmt.execute("DROP DATABASE " + database); + } + } catch (Exception e) { + if (e.toString().contains("Catalog Version Mismatch: A DDL occurred while processing")) { + continue; + } else { + throw e; + } + } + } + } + } + + @AfterClass + public static void tearDownAfter() throws Exception { + // Close the root connection, which is not cleaned up after each test. + if (connection != null && !connection.isClosed()) { + connection.close(); + } + pgInitialized = false; + LOG.info("Destroying mini-cluster"); + if (miniCluster != null) { + destroyMiniCluster(); + miniCluster = null; + } + } + + protected static class Row implements Comparable, Cloneable { + static Row fromResultSet(ResultSet rs) throws SQLException { + List elems = new ArrayList<>(); + List columnNames = new ArrayList<>(); + for (int i = 1; i <= rs.getMetaData().getColumnCount(); i++) { + elems.add(rs.getObject(i)); + columnNames.add(rs.getMetaData().getColumnLabel(i)); + } + // Pre-initialize stuff while connection is still available. + for (Object el : elems) { + if (el instanceof PgArray) + ((PgArray) el).getArray(); + } + return new Row(elems, columnNames); + } + + List elems = new ArrayList<>(); + + /** + * List of column names, should have the same size as {@link #elems}. + *

+ * Not used for equality, hash code and comparison. + */ + List columnNames = new ArrayList<>(); + + Row(List elems, List columnNames) { + checkArgument(elems.size() == columnNames.size()); + this.elems = elems; + this.columnNames = columnNames; + } + + /** Returns a column name if available, or {@code null} otherwise. */ + String getColumnName(int index) { + return columnNames.size() > 0 ? columnNames.get(index) : null; + } + + String getString(int index) { + return (String) elems.get(index); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof Row)) { + return false; + } + Row other = (Row)obj; + return compareTo(other) == 0; + } + + @Override + public int compareTo(Row that) { + // In our test, if selected Row has different number of columns from expected row, something + // must be very wrong. Stop the test here. + assertEquals("Row width mismatch between " + this + " and " + that, + this.elems.size(), that.elems.size()); + return compare(this.elems, that.elems); + } + + @Override + public int hashCode() { + return elems.hashCode(); + } + + @Override + public String toString() { + return toString(false /* printColumnNames */); + } + + public String toString(boolean printColumnNames) { + StringBuilder sb = new StringBuilder(); + sb.append("Row["); + for (int i = 0; i < elems.size(); i++) { + if (i > 0) sb.append(','); + if (printColumnNames) { + String columnNameOrNull = getColumnName(i); + sb.append((columnNameOrNull != null ? columnNameOrNull : i) + "="); + } + if (elems.get(i) == null) { + sb.append("null"); + } else { + sb.append(elems.get(i).getClass().getName() + "::"); + sb.append(elems.get(i).toString()); + } + } + sb.append(']'); + return sb.toString(); + } + + @Override + public Row clone() { + try { + Row clone = (Row) super.clone(); + clone.elems = new ArrayList<>(this.elems); + clone.columnNames = new ArrayList<>(this.columnNames); + return clone; + } catch (CloneNotSupportedException ex) { + // Not possible + throw new RuntimeException(ex); + } + } + + /** + * Compare two objects if possible. Is able to compare: + *
    + *
  • Primitives + *
  • Comparables (including {@code String}) - but cannot handle the case of two unrelated + * Comparables + *
  • {@code PGobject}s wrapping {@code Comparable}s + *
  • Arrays, {@code PgArray}s or lists of the above types + *
+ */ + @SuppressWarnings({ "unchecked", "rawtypes" }) + private static int compare(Object o1, Object o2) { + if (o1 == null || o2 == null) { + if (o1 != o2) { + return o1 == null ? -1 : 1; + } + return 0; + } else { + Object promoted1 = promoteType(o1); + Object promoted2 = promoteType(o2); + if (promoted1 instanceof Long && promoted2 instanceof Long) { + return ((Long) promoted1).compareTo((Long) promoted2); + } else if (promoted1 instanceof Double && promoted2 instanceof Double) { + return ((Double) promoted1).compareTo((Double) promoted2); + } else if (promoted1 instanceof Number && promoted2 instanceof Number) { + return Double.compare( + ((Number) promoted1).doubleValue(), + ((Number) promoted2).doubleValue()); + } else if (promoted1 instanceof Comparable && promoted2 instanceof Comparable) { + // This is unsafe but we dont expect arbitrary types here. + return ((Comparable) promoted1).compareTo((Comparable) promoted2); + } else if (promoted1 instanceof List && promoted2 instanceof List) { + List list1 = (List) promoted1; + List list2 = (List) promoted2; + if (list1.size() != list2.size()) { + return Integer.compare(list1.size(), list2.size()); + } + for (int i = 0; i < list1.size(); ++i) { + int comparisonResult = compare(list1.get(i), list2.get(i)); + if (comparisonResult != 0) { + return comparisonResult; + } + } + return 0; + } else { + throw new IllegalArgumentException("Cannot compare " + + o1 + " (of class " + o1.getClass().getCanonicalName() + ") with " + + o2 + " (of class " + o1.getClass().getCanonicalName() + ")"); + } + } + } + + /** Converts the value to a widest one of the same type for comparison */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + private static Object promoteType(Object v) { + if (v instanceof Byte || v instanceof Short || v instanceof Integer) { + return ((Number)v).longValue(); + } else if (v instanceof Float) { + return ((Float)v).doubleValue(); + } else if (v instanceof Comparable) { + return v; + } else if (v instanceof List) { + return v; + } else if (v instanceof PGobject) { + return promoteType(((PGobject) v).getValue()); // For PG_LSN type. + } else if (v instanceof PgArray) { + try { + return promoteType(((PgArray) v).getArray()); + } catch (SQLException ex) { + throw new RuntimeException("SQL exception during type promotion", ex); + } + } else if (v.getClass().isArray()) { + List list = new ArrayList<>(); + // Unfortunately there's no easy way to automate that, we have to enumerate all array types + // explicitly. + if (v instanceof byte[]) { + for (byte ve : (byte[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof short[]) { + for (short ve : (short[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof int[]) { + for (int ve : (int[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof long[]) { + for (long ve : (long[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof float[]) { + for (float ve : (float[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof double[]) { + for (double ve : (double[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof boolean[]) { + for (boolean ve : (boolean[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof char[]) { + for (char ve : (char[]) v) { + list.add(promoteType(ve)); + } + } else if (v instanceof Object[]) { + for (Object ve : (Object[]) v) { + list.add(promoteType(ve)); + } + } + return list; + } else { + throw new IllegalArgumentException(v + " (of class " + v.getClass().getSimpleName() + ")" + + " cannot be promoted for comparison!"); + } + } + } + + protected List getRowList(Statement stmt, String query) throws SQLException { + try (ResultSet rs = stmt.executeQuery(query)) { + return getRowList(rs); + } + } + + protected static List getRowList(ResultSet rs) throws SQLException { + List rows = new ArrayList<>(); + while (rs.next()) { + rows.add(Row.fromResultSet(rs)); + } + return rows; + } + + @Override + public int getTestMethodTimeoutSec() { + // initdb takes a really long time on macOS in debug mode. + return 1200; + } + + /** Run a process, returning output lines. */ + protected List runProcess(String... args) throws Exception { + return runProcess(new ProcessBuilder(args)); + } + + /** Run a process, returning output lines. */ + protected List runProcess(ProcessBuilder procBuilder) throws Exception { + Process proc = procBuilder.start(); + int code = proc.waitFor(); + if (code != 0) { + String err = IOUtils.toString(proc.getErrorStream(), StandardCharsets.UTF_8); + fail("Process exited with code " + code + ", message: <" + err.trim() + ">"); + } + String output = IOUtils.toString(proc.getInputStream(), StandardCharsets.UTF_8); + return Arrays.asList(output.split("\n")); + } + + protected void setServerFlag(HostAndPort server, String flag, String value) throws Exception { + runProcess(TestUtils.findBinary("yb-ts-cli"), + "--server_address", + server.toString(), + "set_flag", + "-force", + flag, + value); + } + + /** + * Create a DB Stream ID using yb-admin tool + * @param masterAddresses + * @param namespaceName + * @return DB Stream ID + * @throws Exception + */ + protected String createDBStreamUsingYbAdmin(String masterAddresses, + String namespaceName) throws Exception { + // the command only returns one line so it is safe to fetch the line at index 0 + String outputLine = runProcess(TestUtils.findBinary("yb-admin"), + "-master_addresses", + masterAddresses, + "create_change_data_stream", + "ysql."+namespaceName).get(0); + + if (outputLine.toLowerCase(Locale.ROOT).contains("error")) { + throw new RuntimeException("Error while creating DB Stream"); + } + + // getting the stream id from this line + String[] splitRes = outputLine.split(":"); + + // the last element after splitting will be the stream ID + return splitRes[splitRes.length - 1].trim(); + } + + /** + * Delete a created DB stream ID using the yb-admin tool + * @param masterAddresses + * @param dbStreamId + * @return The deleted stream ID + * @throws Exception + */ + protected String deleteDBStreamUsingYbAdmin(String masterAddresses, + String dbStreamId) throws Exception { + // the output contains one line only + String outputLine = runProcess(TestUtils.findBinary("yb-admin"), + "-master_addresses", + masterAddresses, + "delete_change_data_stream", + dbStreamId).get(0); + + if (outputLine.toLowerCase(Locale.ROOT).contains("error")) { + throw new RuntimeException("Error while deleting DB Stream"); + } + + // getting the stream ID from this line, the stream ID will be the last element after the split + String[] splitRes = outputLine.split(":"); + + return splitRes[splitRes.length - 1].trim(); + } + + public static class ConnectionBuilder implements Cloneable { + private static final int MAX_CONNECTION_ATTEMPTS = 15; + private static final int INITIAL_CONNECTION_DELAY_MS = 500; + + private final MiniYBCluster miniCluster; + + private int tserverIndex = 0; + private String database = DEFAULT_PG_DATABASE; + private String user = DEFAULT_PG_USER; + private String password = null; + private String preferQueryMode = null; + private String sslmode = null; + private String sslcert = null; + private String sslkey = null; + private String sslrootcert = null; + + ConnectionBuilder(MiniYBCluster miniCluster) { + this.miniCluster = checkNotNull(miniCluster); + } + + ConnectionBuilder withTServer(int tserverIndex) { + ConnectionBuilder copy = clone(); + copy.tserverIndex = tserverIndex; + return copy; + } + + ConnectionBuilder withDatabase(String database) { + ConnectionBuilder copy = clone(); + copy.database = database; + return copy; + } + + ConnectionBuilder withUser(String user) { + ConnectionBuilder copy = clone(); + copy.user = user; + return copy; + } + + ConnectionBuilder withPassword(String password) { + ConnectionBuilder copy = clone(); + copy.password = password; + return copy; + } + + + ConnectionBuilder withPreferQueryMode(String preferQueryMode) { + ConnectionBuilder copy = clone(); + copy.preferQueryMode = preferQueryMode; + return copy; + } + + ConnectionBuilder withSslMode(String sslmode) { + ConnectionBuilder copy = clone(); + copy.sslmode = sslmode; + return copy; + } + + ConnectionBuilder withSslCert(String sslcert) { + ConnectionBuilder copy = clone(); + copy.sslcert = sslcert; + return copy; + } + + ConnectionBuilder withSslKey(String sslkey) { + ConnectionBuilder copy = clone(); + copy.sslkey = sslkey; + return copy; + } + + ConnectionBuilder withSslRootCert(String sslrootcert) { + ConnectionBuilder copy = clone(); + copy.sslrootcert = sslrootcert; + return copy; + } + + @Override + protected ConnectionBuilder clone() { + try { + return (ConnectionBuilder) super.clone(); + } catch (CloneNotSupportedException ex) { + throw new RuntimeException("This can't happen, but to keep compiler happy", ex); + } + } + + Connection connect() throws Exception { + final InetSocketAddress postgresAddress = miniCluster.getPostgresContactPoints() + .get(tserverIndex); + String url = String.format( + "jdbc:yugabytedb://%s:%d/%s", + postgresAddress.getHostName(), + postgresAddress.getPort(), + database + ); + + Properties props = new Properties(); + props.setProperty("user", user); + if (password != null) { + props.setProperty("password", password); + } + if (preferQueryMode != null) { + props.setProperty("preferQueryMode", preferQueryMode); + } + if (sslmode != null) { + props.setProperty("sslmode", sslmode); + } + if (sslcert != null) { + props.setProperty("sslcert", sslcert); + } + if (sslkey != null) { + props.setProperty("sslkey", sslkey); + } + if (sslrootcert != null) { + props.setProperty("sslrootcert", sslrootcert); + } + if (EnvAndSysPropertyUtil.isEnvVarOrSystemPropertyTrue("YB_PG_JDBC_TRACE_LOGGING")) { + props.setProperty("loggerLevel", "TRACE"); + } + + int delayMs = INITIAL_CONNECTION_DELAY_MS; + for (int attempt = 1; attempt <= MAX_CONNECTION_ATTEMPTS; ++attempt) { + Connection connection = null; + try { + connection = checkNotNull(DriverManager.getConnection(url, props)); + connection.setAutoCommit(true); + + return connection; + } catch (SQLException sqlEx) { + // Close the connection now if we opened it, instead of waiting until the end of the test. + if (connection != null) { + try { + connection.close(); + } catch (SQLException closingError) { + LOG.error("Failure to close connection during failure cleanup before a retry:", + closingError); + LOG.error("When handling this exception when opening/setting up connection:", sqlEx); + } + } + + boolean retry = false; + + if (attempt < MAX_CONNECTION_ATTEMPTS) { + if (sqlEx.getMessage().contains("FATAL: the database system is starting up") + || sqlEx.getMessage().contains("refused. Check that the hostname and port are " + + "correct and that the postmaster is accepting")) { + retry = true; + + LOG.info("Postgres is still starting up, waiting for " + delayMs + " ms. " + + "Got message: " + sqlEx.getMessage()); + } else if (sqlEx.getMessage().contains("the database system is in recovery mode")) { + retry = true; + + LOG.info("Postgres is in recovery mode, waiting for " + delayMs + " ms. " + + "Got message: " + sqlEx.getMessage()); + } + } + + if (retry) { + Thread.sleep(delayMs); + delayMs = Math.min(delayMs + 500, 10000); + } else { + LOG.error("Exception while trying to create connection (after " + attempt + + " attempts): " + sqlEx.getMessage()); + throw sqlEx; + } + } + } + throw new IllegalStateException("Should not be able to reach here"); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCWithYugabyted.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCWithYugabyted.java new file mode 100644 index 000000000000..8e18a6e1a3e1 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/CDCWithYugabyted.java @@ -0,0 +1,60 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import com.google.common.net.HostAndPort; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.yb.cdc.util.TestUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; + +/** + * To test with Yugabyted, all the user need to do is extend CDCWithYugabyted instead of + * CDCBaseClass + *

+ * So instead of the below line
+ * public class XYZ extends CDCBaseClass
+ * write
+ * public class XYZ extends CDCWithYugabyted + */ +public class CDCWithYugabyted { + protected Connection connection; + protected Statement statement; + + @BeforeClass + public static void startYugabyte() { + TestUtils.startYugabyted(); + } + + @AfterClass + public static void destroyYugabyte() { + TestUtils.destroyYugabyted(); + } + + public void setUp() throws Exception { + connection = DriverManager.getConnection("jdbc:yugabytedb://127.0.0.1:5433/yugabyte?" + + "user=yugabyte&password=yugabyte"); + } + + protected String getMasterAddresses() { + return "127.0.0.1:7100"; + } + + protected HostAndPort getTserverHostAndPort() { + return HostAndPort.fromString("127.0.0.1:9000"); + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecord3Proto.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecord3Proto.java new file mode 100644 index 000000000000..9e4ed3f55c11 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecord3Proto.java @@ -0,0 +1,114 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import static org.yb.AssertionWrappers.*; +import java.util.Objects; + +public class ExpectedRecord3Proto { + public int col1; + public int col2; + public int col3; + public Op opType; + + public ExpectedRecord3Proto(int col1, int col2, int col3, Op opType) { + this.col1 = col1; + this.col2 = col2; + this.col3 = col3; + this.opType = opType; + } + + private static void checkInsertRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecord3Proto expectedRecord) { + assertEquals(Op.INSERT, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, + record.getRowMessage().getNewTuple(1).getDatumInt32()); + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDeleteRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecord3Proto expectedRecord) { + assertEquals(Op.DELETE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, + record.getRowMessage().getOldTuple(0).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkBeginRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.BEGIN, record.getRowMessage().getOp()); + } + + private static void checkCommitRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.COMMIT, record.getRowMessage().getOp()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDDLRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.DDL, record.getRowMessage().getOp()); + assertTrue(record.getRowMessage().hasSchema()); + assertTrue(record.getRowMessage().hasPgschemaName()); + } + + private static void checkUpdateRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecord3Proto expectedRecord) { + assertEquals(Op.UPDATE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + + if (record.getRowMessage().getNewTupleCount() == 2) { + if (Objects.equals(record.getRowMessage().getNewTuple(1).getColumnName(), "b")) { + assertEquals(expectedRecord.col2, + record.getRowMessage().getNewTuple(1).getDatumInt32()); + } else if (Objects.equals(record.getRowMessage().getNewTuple(1).getColumnName(), "c")) { + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(1).getDatumInt32()); + } + } else if (record.getRowMessage().getNewTupleCount() == 3) { + assertEquals(expectedRecord.col2, + record.getRowMessage().getNewTuple(1).getDatumInt32()); + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + } + + assertTrue(record.hasCdcSdkOpId()); + } + + public static void checkRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecord3Proto expectedRecord) { + switch (expectedRecord.opType) { + case INSERT: + checkInsertRecord(record, expectedRecord); + break; + case DELETE: + checkDeleteRecord(record, expectedRecord); + break; + case UPDATE: + checkUpdateRecord(record, expectedRecord); + break; + case BEGIN: + checkBeginRecord(record); + break; + case COMMIT: + checkCommitRecord(record); + break; + case DDL: + checkDDLRecord(record); + break; + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordCPKProto.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordCPKProto.java new file mode 100644 index 000000000000..b01832a6068a --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordCPKProto.java @@ -0,0 +1,120 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import static org.yb.AssertionWrappers.*; +import java.util.Objects; + +public class ExpectedRecordCPKProto { + public int col1; + public int col2; + public int col3; + public int col4; + public Op opType; + + public ExpectedRecordCPKProto(int col1, int col2, int col3, int col4, Op opType) { + this.col1 = col1; + this.col2 = col2; + this.col3 = col3; + this.col4 = col4; + this.opType = opType; + } + + private static void checkInsertRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordCPKProto expectedRecord) { + assertEquals(Op.INSERT, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, + record.getRowMessage().getNewTuple(1).getDatumInt32()); + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + assertEquals(expectedRecord.col4, + record.getRowMessage().getNewTuple(3).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDeleteRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordCPKProto expectedRecord) { + assertEquals(Op.DELETE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, + record.getRowMessage().getOldTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, + record.getRowMessage().getOldTuple(1).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkBeginRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.BEGIN, record.getRowMessage().getOp()); + } + + private static void checkCommitRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.COMMIT, record.getRowMessage().getOp()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDDLRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.DDL, record.getRowMessage().getOp()); + assertTrue(record.getRowMessage().hasSchema()); + assertTrue(record.getRowMessage().hasPgschemaName()); + } + + private static void checkUpdateRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordCPKProto expectedRecord) { + assertEquals(Op.UPDATE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + + if (record.getRowMessage().getNewTupleCount() == 3) { + if (Objects.equals(record.getRowMessage().getNewTuple(1).getColumnName(), "c")) { + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + } else if (Objects.equals(record.getRowMessage().getNewTuple(1).getColumnName(), "d")) { + assertEquals(expectedRecord.col4, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + } + } else if (record.getRowMessage().getNewTupleCount() == 4) { + assertEquals(expectedRecord.col3, + record.getRowMessage().getNewTuple(2).getDatumInt32()); + assertEquals(expectedRecord.col4, + record.getRowMessage().getNewTuple(3).getDatumInt32()); + } + + assertTrue(record.hasCdcSdkOpId()); + } + + public static void checkRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordCPKProto expectedRecord) { + switch (expectedRecord.opType) { + case INSERT: + checkInsertRecord(record, expectedRecord); + break; + case DELETE: + checkDeleteRecord(record, expectedRecord); + break; + case UPDATE: + checkUpdateRecord(record, expectedRecord); + break; + case BEGIN: + checkBeginRecord(record); + break; + case COMMIT: + checkCommitRecord(record); + break; + case DDL: + checkDDLRecord(record); + break; + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYCQLGeneric.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYCQLGeneric.java new file mode 100644 index 000000000000..2ba144825d17 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYCQLGeneric.java @@ -0,0 +1,121 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import org.yb.Value; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; + +import static org.yb.AssertionWrappers.*; + +public class ExpectedRecordYCQLGeneric { + public int col1; + public T col2; + public Op opType; + + private static final int columnCount = 2; + + public static Object getValue(CdcService.CDCSDKProtoRecordPB record) { + return getValue(record, 1); + } + + public static Object getValue(CdcService.CDCSDKProtoRecordPB record, int index) { + // Considering the index 1 since only the second column is going to be varying type here. + Value.DatumMessagePB tuple = record.getRowMessage().getNewTuple(index); + + if (tuple.hasDatumBool()) { + return tuple.getDatumBool(); + } else if (tuple.hasDatumDouble()) { + return tuple.getDatumDouble(); + } else if (tuple.hasDatumFloat()) { + return tuple.getDatumFloat(); + } else if (tuple.hasDatumString()) { + return tuple.getDatumString(); + } else if (tuple.hasDatumInt32()) { + return tuple.getDatumInt32(); + } else if (tuple.hasDatumInt64()) { + return tuple.getDatumInt64(); + } else { + return null; + } + } + + public ExpectedRecordYCQLGeneric(int col1, T col2, Op opType) { + this.col1 = col1; + this.col2 = col2; + this.opType = opType; + } + + private static void checkInsertRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYCQLGeneric expectedRecord) { + assertEquals(Op.INSERT, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, getValue(record)); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDeleteRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYCQLGeneric expectedRecord) { + assertEquals(Op.DELETE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkBeginRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.BEGIN, record.getRowMessage().getOp()); + } + + private static void checkCommitRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.COMMIT, record.getRowMessage().getOp()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDDLRecord(CdcService.CDCSDKProtoRecordPB record, + int numberOfColumns) { + assertEquals(Op.DDL, record.getRowMessage().getOp()); + assertTrue(record.getRowMessage().hasSchema()); + assertEquals(numberOfColumns, record.getRowMessage().getSchema().getColumnInfoCount()); + } + + private static void checkUpdateRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYCQLGeneric expectedRecord) { + assertEquals(Op.UPDATE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, getValue(record)); + assertTrue(record.hasCdcSdkOpId()); + } + + public static void checkRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYCQLGeneric expectedRecord) { + switch (expectedRecord.opType) { + case INSERT: + checkInsertRecord(record, expectedRecord); + break; + case DELETE: + checkDeleteRecord(record, expectedRecord); + break; + case BEGIN: + checkBeginRecord(record); + break; + case COMMIT: + checkCommitRecord(record); + break; + case UPDATE: + checkUpdateRecord(record, expectedRecord); + break; + case DDL: + checkDDLRecord(record, columnCount); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYSQL.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYSQL.java new file mode 100644 index 000000000000..a755a2b81e08 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/ExpectedRecordYSQL.java @@ -0,0 +1,122 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.common; + +import org.yb.Value; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; + +import static org.yb.AssertionWrappers.*; + +public class ExpectedRecordYSQL { + public int col1; + public T col2; + public Op opType; + + private static final int columnCount = 2; + + public ExpectedRecordYSQL(int col1, T col2, Op opType) { + this.col1 = col1; + this.col2 = col2; + this.opType = opType; + } + + public static Object getValue(CdcService.CDCSDKProtoRecordPB record) { + return getValue(record, 1); + } + + public static Object getValue(CdcService.CDCSDKProtoRecordPB record, int index) { + // Considering the index 1 since only the second column is going to be varying type here. + Value.DatumMessagePB tuple = record.getRowMessage().getNewTuple(index); + + if (tuple.hasDatumBool()) { + return tuple.getDatumBool(); + } else if (tuple.hasDatumDouble()) { + return tuple.getDatumDouble(); + } else if (tuple.hasDatumFloat()) { + return tuple.getDatumFloat(); + } else if (tuple.hasDatumString()) { + return tuple.getDatumString(); + } else if (tuple.hasDatumInt32()) { + return tuple.getDatumInt32(); + } else if (tuple.hasDatumInt64()) { + return tuple.getDatumInt64(); + } else { + return null; + } + } + + private static void checkInsertRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYSQL expectedRecord) { + assertEquals(Op.INSERT, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, getValue(record)); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDeleteRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYSQL expectedRecord) { + assertEquals(Op.DELETE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getOldTuple(0).getDatumInt32()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkBeginRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.BEGIN, record.getRowMessage().getOp()); + } + + private static void checkCommitRecord(CdcService.CDCSDKProtoRecordPB record) { + assertEquals(Op.COMMIT, record.getRowMessage().getOp()); + assertTrue(record.hasCdcSdkOpId()); + } + + private static void checkDDLRecord(CdcService.CDCSDKProtoRecordPB record, + int numberOfColumns) { + assertEquals(Op.DDL, record.getRowMessage().getOp()); + assertTrue(record.getRowMessage().hasSchema()); + assertTrue(record.getRowMessage().hasPgschemaName()); + assertEquals(numberOfColumns, record.getRowMessage().getSchema().getColumnInfoCount()); + } + + private static void checkUpdateRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYSQL expectedRecord) { + assertEquals(Op.UPDATE, record.getRowMessage().getOp()); + assertEquals(expectedRecord.col1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + assertEquals(expectedRecord.col2, getValue(record)); + assertTrue(record.hasCdcSdkOpId()); + } + + public static void checkRecord(CdcService.CDCSDKProtoRecordPB record, + ExpectedRecordYSQL expectedRecord) { + switch (expectedRecord.opType) { + case INSERT: + checkInsertRecord(record, expectedRecord); + break; + case DELETE: + checkDeleteRecord(record, expectedRecord); + break; + case UPDATE: + checkUpdateRecord(record, expectedRecord); + break; + case BEGIN: + checkBeginRecord(record); + break; + case COMMIT: + checkCommitRecord(record); + break; + case DDL: + checkDDLRecord(record, columnCount); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/common/HelperValues.java b/java/yb-cdc/src/test/java/org/yb/cdc/common/HelperValues.java new file mode 100644 index 000000000000..aa8f930a4b9e --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/common/HelperValues.java @@ -0,0 +1,171 @@ +package org.yb.cdc.common; + +public class HelperValues { + public static final String dropAllTablesWithTypes = "drop table if exists testbit, " + + "testboolean, testbox, testbytea, testcidr, testcircle, testdate, testdouble, testinet, " + + "testint, testjson, testjsonb, testline, testlseg, testmacaddr8, testmacaddr, " + + "testmoney, testnumeric, testpath, testpoint, testpolygon, testtext, testtime, " + + "testtimestamp, testtimetz, testuuid, testvarbit, testtstz, testint4range, " + + "testint8range, testtsrange, testtstzrange, testdaterange;"; + + public static final String createTableWithDefaults = "create table testdefault " + + "(a int primary key, " + + "bitval bit(4) default '1111', boolval boolean default TRUE, " + + "boxval box default '(0,0),(1,1)', byteval bytea default E'\\\\001', " + + "cidrval cidr default '10.1.0.0/16', crcl circle default '0,0,5'," + + "dt date default '2000-01-01', dp double precision default 32.34, " + + "inetval inet default '127.0.0.1', i int default 404, " + + "js json default '{\"a\":\"b\"}', jsb jsonb default '{\"a\":\"b\"}', " + + "ln line default '{1,2,-8}', ls lseg default '[(0,0),(2,4)]', " + + "mc8 macaddr8 default '22:00:5c:03:55:08:01:02', mc macaddr default '2C:54:91:88:C9:E3', " + + "mn money default 100, nm numeric default 12.34, " + + "pth path default '(1,2),(20,-10)', pnt point default '(0,0)', " + + "poly polygon default '(1,3),(4,12),(2,4)', txt text default 'default text value', " + + "tm time default '00:00:00', ts timestamp default '2000-09-01 00:00:00', " + + "ttz timetz default '00:00:00+05:30', " + + "u uuid default 'ffffffff-ffff-ffff-ffff-ffffffffffff', " + + "vb varbit(4) default '11', tstz timestamptz default '1970-01-01 00:10:00+05:30', " + + "i4r int4range default '(1,10)', i8r int8range default '(100, 200)', " + + "tsr tsrange default '(1970-01-01 00:00:00, 1970-01-01 12:00:00)', " + + "tstzr tstzrange default '(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)', " + + "dr daterange default '(1970-01-01,2000-01-01)');"; + + public static Object[] expectedDefaultValues = {"1111", true, "(1,1),(0,0)", "\\x01", + "10.1.0.0/16", "<(0,0),5>", "2000-01-01", 32.34, "127.0.0.1", 404, "{\"a\":\"b\"}", + "{\"a\": \"b\"}", "{1,2,-8}", "[(0,0),(2,4)]", "22:00:5c:03:55:08:01:02", + "2c:54:91:88:c9:e3", "$100.00", 12.34, "((1,2),(20,-10))", "(0,0)", "((1,3),(4,12),(2,4))", + "default text value", "00:00:00", "2000-09-01 00:00:00", "00:00:00+05:30", + "ffffffff-ffff-ffff-ffff-ffffffffffff", "11", "1969-12-31 18:40:00+00", "[2,10)", + "[101,200)", "(\"1970-01-01 00:00:00\",\"1970-01-01 12:00:00\")", + "(\"2017-07-04 12:30:30+00\",\"2021-07-04 07:00:30+00\")", "[1970-01-02,2000-01-01)", + ""}; + + public static String insertionTemplateForArrays = "insert into %s values (1, %s, %s, %s, %s, " + + "%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, " + + "%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s);"; + + public static String dropAllArrayTables = "drop table if exists testvb, " + + "testboolval, testchval, testvchar, testdt, testdp, testinetval, testintval, testjsonval, " + + "testjsonbval, testmac, testmac8, testmoneyval, testrl, testsi, testtextval, " + + "testtval, testttzval, testtimestampval, testtimestamptzval, testu, testi4r, testi8r, " + + "testdr, testtsr, testtstzr, testnr, testbx, testln, testls, testpt, testcrcl, " + + "testpoly, testpth, testinterv, testcidrval, testtxid"; + + public static String createTableWithMultiDimensionalArrayColumns = "create table testmulti " + + "(a int primary key, vb varbit(10)[], boolval boolean[], chval char(5)[], " + + "vchar varchar(20)[], dt date[], " + "dp double precision[], " + + "inetval inet[], intval integer[], jsonval json[], jsonbval jsonb[], mac macaddr[], " + + "mac8 macaddr8[], moneyval money[], rl real[], si smallint[], textval text[], " + + "tval time[], ttzval timetz[], timestampval timestamp[], timestamptzcal timestamptz[], " + + "u uuid[], i4r int4range[], i8r int8range[], dr daterange[], tsr tsrange[], " + + "tstzr tstzrange[], nr numrange[], bx box[], ln line[], ls lseg[], pt point[], " + + "crcl circle[], poly polygon[], pth path[], interv interval[], cidrval cidr[], " + + "txid txid_snapshot[]);"; + + public static String createTableWithSingleDimensionalArrayColumns = "create table testsingle " + + "(a int primary key, vb varbit(10)[], boolval boolean[], chval char(5)[], " + + "vchar varchar(20)[], dt date[], " + "dp double precision[], " + + "inetval inet[], intval integer[], jsonval json[], jsonbval jsonb[], mac macaddr[], " + + "mac8 macaddr8[], moneyval money[], rl real[], si smallint[], textval text[], " + + "tval time[], ttzval timetz[], timestampval timestamp[], timestamptzcal timestamptz[], " + + "u uuid[], i4r int4range[], i8r int8range[], dr daterange[], tsr tsrange[], " + + "tstzr tstzrange[], nr numrange[], bx box[], ln line[], ls lseg[], pt point[], " + + "crcl circle[], poly polygon[], pth path[], interv interval[], cidrval cidr[], " + + "txid txid_snapshot[]);"; + + public static String[] expectedMultiDimensionalArrayColumnRecords = { + "{{1011,011101,1101110111},{1011,011101,1101110111}}", + "{{f,t,t,f},{f,t,t,f}}", + "{{five5,five5},{five5,five5}}", + "{{\"sample varchar\",\"test string\"},{\"sample varchar\",\"test string\"}}", + "{{2021-10-07,1970-01-01},{2021-10-07,1970-01-01}}", + "{{1.23,2.34,3.45},{1.23,2.34,3.45}}", + "{{127.0.0.1,192.168.1.1},{127.0.0.1,192.168.1.1}}", + "{{1,2,3},{1,2,3}}", + "{{\"{\\\"a\\\":\\\"b\\\"}\",\"{\\\"c\\\":\\\"d\\\"}\"}," + + "{\"{\\\"a\\\":\\\"b\\\"}\",\"{\\\"c\\\":\\\"d\\\"}\"}}", + "{{\"{\\\"a\\\": \\\"b\\\"}\",\"{\\\"c\\\": \\\"d\\\"}\"}," + + "{\"{\\\"a\\\": \\\"b\\\"}\",\"{\\\"c\\\": \\\"d\\\"}\"}}", + "{{2c:54:91:88:c9:e3,2c:b8:01:76:c9:e3,2c:54:f1:88:c9:e3}," + + "{2c:54:91:88:c9:e3,2c:b8:01:76:c9:e3,2c:54:f1:88:c9:e3}}", + "{{22:00:5c:03:55:08:01:02,22:10:5c:03:55:d8:f1:02}," + + "{22:00:5c:03:55:08:01:02,22:10:5c:03:55:d8:f1:02}}", + "{{$100.55,$200.50,$50.05},{$100.55,$200.50,$50.05}}", + "{{1.23,4.56,7.8901},{1.23,4.56,7.8901}}", + "{{1,2,3,4,5,6},{1,2,3,4,5,6}}", + "{{sample1,sample2},{sample1,sample2}}", + "{{12:00:32,22:10:20,23:59:59,00:00:00},{12:00:32,22:10:20,23:59:59,00:00:00}}", + "{{11:00:00+05:30,23:00:59+00,09:59:00+00},{11:00:00+05:30,23:00:59+00,09:59:00+00}}", + "{{\"1970-01-01 00:00:10\",\"2000-01-01 00:00:10\"}," + + "{\"1970-01-01 00:00:10\",\"2000-01-01 00:00:10\"}}", + "{{\"1969-12-31 18:30:10+00\",\"2000-01-01 00:00:10+00\"}," + + "{\"1969-12-31 18:30:10+00\",\"2000-01-01 00:00:10+00\"}}", + "{{123e4567-e89b-12d3-a456-426655440000,123e4567-e89b-12d3-a456-426655440000}," + + "{123e4567-e89b-12d3-a456-426655440000,123e4567-e89b-12d3-a456-426655440000}}", + "{{\"[2,5)\",\"[11,100)\"},{\"[2,5)\",\"[11,100)\"}}", + "{{\"[2,10)\",\"[901,10000)\"},{\"[2,10)\",\"[901,10000)\"}}", + "{{\"[2000-09-21,2021-10-08)\",\"[1970-01-02,2000-01-01)\"}," + + "{\"[2000-09-21,2021-10-08)\",\"[1970-01-02,2000-01-01)\"}}", + "{{\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"," + + "\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"}," + + "{\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"," + + "\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"}}", + "{{\"(\\\"2017-07-04 12:30:30+00\\\",\\\"2021-07-04 07:00:30+00\\\")\"," + + "\"(\\\"1970-09-14 12:30:30+00\\\",\\\"2021-10-13 04:02:30+00\\\")\"}," + + "{\"(\\\"2017-07-04 12:30:30+00\\\",\\\"2021-07-04 07:00:30+00\\\")\"," + + "\"(\\\"1970-09-14 12:30:30+00\\\",\\\"2021-10-13 04:02:30+00\\\")\"}}", + "{{\"(10.42,11.354)\",\"(-0.99,100.9)\"},{\"(10.42,11.354)\",\"(-0.99,100.9)\"}}", + "{{(8,9),(1,3);(9,27),(-1,-1)};{(8,9),(1,3);(9,27),(-1,-1)}}", + "{{\"{2.5,-1,0}\",\"{1,2,-10}\"},{\"{2.5,-1,0}\",\"{1,2,-10}\"}}", + "{{\"[(0,0),(2,5)]\",\"[(0,5),(6,2)]\"},{\"[(0,0),(2,5)]\",\"[(0,5),(6,2)]\"}}", + "{{\"(1,2)\",\"(10,11.5)\",\"(0,-1)\"},{\"(1,2)\",\"(10,11.5)\",\"(0,-1)\"}}", + "{{\"<(1,2),4>\",\"<(-1,0),5>\"},{\"<(1,2),4>\",\"<(-1,0),5>\"}}", + "{{\"((1,3),(4,12),(2,4))\",\"((1,-1),(4,-12),(-2,-4))\"}," + + "{\"((1,3),(4,12),(2,4))\",\"((1,-1),(4,-12),(-2,-4))\"}}", + "{{\"((1,2),(10,15),(0,0))\",\"((1,2),(10,15),(10,0),(-3,-2))\"}," + + "{\"((1,2),(10,15),(0,0))\",\"((1,2),(10,15),(10,0),(-3,-2))\"}}", + "{{01:16:06.2,\"29 days\"},{01:16:06.2,\"29 days\"}}", + "{{12.2.0.0/22,10.1.0.0/16},{12.2.0.0/22,10.1.0.0/16}}", + "{{3:3:,3:3:},{3:3:,3:3:}}"}; + + public static String[] expectedSingleDimensionalArrayColumnRecords = { + "{1011,011101,1101110111}", + "{f,t,t,f}", + "{five5,five5}", + "{\"sample varchar\",\"test string\"}", + "{2021-10-07,1970-01-01}", + "{1.23,2.34,3.45}", + "{127.0.0.1,192.168.1.1}", + "{1,2,3}", + "{\"{\\\"a\\\":\\\"b\\\"}\",\"{\\\"c\\\":\\\"d\\\"}\"}", + "{\"{\\\"a\\\": \\\"b\\\"}\",\"{\\\"c\\\": \\\"d\\\"}\"}", + "{2c:54:91:88:c9:e3,2c:b8:01:76:c9:e3,2c:54:f1:88:c9:e3}", + "{22:00:5c:03:55:08:01:02,22:10:5c:03:55:d8:f1:02}", + "{$100.55,$200.50,$50.05}", + "{1.23,4.56,7.8901}", + "{1,2,3,4,5,6}", + "{sample1,sample2}", + "{12:00:32,22:10:20,23:59:59,00:00:00}", + "{11:00:00+05:30,23:00:59+00,09:59:00+00}", + "{\"1970-01-01 00:00:10\",\"2000-01-01 00:00:10\"}", + "{\"1969-12-31 18:30:10+00\",\"2000-01-01 00:00:10+00\"}", + "{123e4567-e89b-12d3-a456-426655440000,123e4567-e89b-12d3-a456-426655440000}", + "{\"[2,5)\",\"[11,100)\"}", + "{\"[2,10)\",\"[901,10000)\"}", + "{\"[2000-09-21,2021-10-08)\",\"[1970-01-02,2000-01-01)\"}", + "{\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"," + + "\"(\\\"1970-01-01 00:00:00\\\",\\\"2000-01-01 12:00:00\\\")\"}", + "{\"(\\\"2017-07-04 12:30:30+00\\\",\\\"2021-07-04 07:00:30+00\\\")\"," + + "\"(\\\"1970-09-14 12:30:30+00\\\",\\\"2021-10-13 04:02:30+00\\\")\"}", + "{\"(10.42,11.354)\",\"(-0.99,100.9)\"}", + "{(8,9),(1,3);(9,27),(-1,-1)}", + "{\"{2.5,-1,0}\",\"{1,2,-10}\"}", + "{\"[(0,0),(2,5)]\",\"[(0,5),(6,2)]\"}", + "{\"(1,2)\",\"(10,11.5)\",\"(0,-1)\"}", + "{\"<(1,2),4>\",\"<(-1,0),5>\"}", + "{\"((1,3),(4,12),(2,4))\",\"((1,-1),(4,-12),(-2,-4))\"}", + "{\"((1,2),(10,15),(0,0))\",\"((1,2),(10,15),(10,0),(-3,-2))\"}", + "{01:16:06.2,\"29 days\"}", + "{12.2.0.0/22,10.1.0.0/16}", + "{3:3:,3:3:}"}; +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriber.java b/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriber.java new file mode 100644 index 000000000000..fea19f9be1d0 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriber.java @@ -0,0 +1,503 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.util; + +import org.apache.log4j.Logger; +import org.yb.cdc.CdcService; +import org.yb.client.*; +import org.yb.master.MasterDdlOuterClass; +import org.yb.master.MasterReplicationOuterClass; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Objects; + +import static org.yb.AssertionWrappers.fail; + +public class CDCSubscriber { + private final static Logger LOG = Logger.getLogger(CDCSubscriber.class); + + private String namespaceName = "yugabyte"; + private String tableName; + private String masterAddrs; + + private String FORMAT; + + private String dbStreamId; + private YBTable table; + private String tableId; + + private YBClient syncClient; + + private String tabletId; + + private Checkpoint checkpoint; + + /** + * This is the default number of tablets as specified in AsyncYBClient + * @see AsyncYBClient + */ + private int numberOfTablets = 10; + + public CDCSubscriber() { + } + + public CDCSubscriber(String masterAddrs) { + this.masterAddrs = masterAddrs; + } + + public CDCSubscriber(String tableName, String masterAddrs) { + this.tableName = tableName; + this.masterAddrs = masterAddrs; + } + + public CDCSubscriber(String namespaceName, String tableName, String masterAddrs) { + this.namespaceName = namespaceName; + this.tableName = tableName; + this.masterAddrs = masterAddrs; + } + + /** + * Getter function to access the YBClient for this subscriber. + */ + public YBClient getSyncClient() { + if (syncClient == null) { + syncClient = createSyncClientForTest(); + } + return syncClient; + } + + /** Only for test purposes. */ + public void setDbStreamId(String dbStreamId) { + this.dbStreamId = dbStreamId; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void setNumberOfTablets(int numberOfTablets) { + this.numberOfTablets = numberOfTablets; + } + + /** + * This is used to set the format of the record to be streamed via CDC. + * + * @param recordFormat Format of the record to be streamed ("json"/"proto") + */ + private void setFormat (String recordFormat) { + FORMAT = recordFormat; + } + + /** + * Creates and returns a YBClient with the specified parameters. + * + * @see AsyncYBClient + * @see YBClient + */ + private YBClient createSyncClientForTest() { + if (syncClient != null) { + return syncClient; + } + + long ADMIN_OPERATION_TIMEOUT = 30000; + long SOCKET_READ_TIMEOUT = 30000; + long OPERATION_TIMEOUT = 30000; + + if (masterAddrs == null) { + masterAddrs = "127.0.0.1:7100"; + } + + LOG.info(String.format("Creating new YBClient with master address: %s", masterAddrs)); + + AsyncYBClient asyncClient = new AsyncYBClient.AsyncYBClientBuilder(masterAddrs) + .defaultAdminOperationTimeoutMs(ADMIN_OPERATION_TIMEOUT) + .defaultOperationTimeoutMs(OPERATION_TIMEOUT) + .defaultSocketReadTimeoutMs(SOCKET_READ_TIMEOUT) + .numTablets(numberOfTablets) + .build(); + + return new YBClient(asyncClient); + } + + /** + * This function uses the created YBClient to get the tableID of the provided table name. + * + * @see YBClient + * @see ListTablesResponse + * @return TableId of provided table name + */ + public String getTableId() throws Exception { + if (syncClient == null) { + syncClient = createSyncClientForTest(); + } + + if (tableId != null) { + return tableId; + } + + ListTablesResponse tablesResp = syncClient.getTablesList(); + + String tId = ""; + String TABLE_NAME = (tableName == null || tableName.isEmpty()) ? "test" : tableName; + String SCHEMA = namespaceName; + for (MasterDdlOuterClass.ListTablesResponsePB.TableInfo tableInfo : + tablesResp.getTableInfoList()) { + if (tableInfo.getName().equals(TABLE_NAME) && + tableInfo.getNamespace().getName().equals(SCHEMA)) { + tId = tableInfo.getId().toStringUtf8(); + } + } + if (tId == null) { + LOG.error(String.format("Could not find a table with name %s.%s", SCHEMA, TABLE_NAME)); + System.exit(0); + } + return tId; + } + + /** + * This function uses the created YBClient and returns the table pertaining to the corresponding + * table ID. + * + * @see YBTable + * @return table of type YBTable corresponding to the provided tableId + */ + public YBTable getTable() throws Exception { + if (table != null) { + return table; + } + + if (tableId == null) { + tableId = getTableId(); + } + + return syncClient.openTableByUUID(tableId); + } + + /** + * This function is the starting point to directly call the getChangesResponse() in order to + * get the records streamed across CDC. It creates a YBClient first, then gets the tableId and + * then the exact table corresponding to that tableId. + * + */ + private void createStreamUtil(String cpType) throws Exception { + syncClient = createSyncClientForTest(); + tableId = getTableId(); + table = getTable(); + + String checkpointingType; + + if (cpType == null || cpType.isEmpty()) { + checkpointingType = "IMPLICIT"; + } else { + checkpointingType = cpType; + } + + LOG.debug("Checkpointing type is: " + checkpointingType); + if (dbStreamId == null || dbStreamId.isEmpty()) { + LOG.debug("Creating a new CDC DB stream"); + dbStreamId = syncClient.createCDCStream(table, "yugabyte", FORMAT, + checkpointingType).getStreamId(); + + LOG.debug(String.format("Created a new DB stream id: %s", dbStreamId)); + } else { + LOG.debug("Using an old cached DB stream id"); + } + + LOG.info("DB Stream id: " + dbStreamId); + + // Calling the GetChangesResponse to make sure it's setting the checkpoint to the beginning. + getResponseFromCDC(new ArrayList<>()); + } + + /** + * This function is used to parse the GetChangesResponse in case of Proto records + * and add it to the provided List in order for further processing. + * + * @param response The GetChangesResponse we get on calling getChangesCDCSDK + * @param records a List to add the records to after parsing it from response + * @see GetChangesResponse + * @see CdcService.CDCSDKProtoRecordPB + * @see List + */ + private void addProtoRecords(GetChangesResponse response, + List records) { + for (CdcService.CDCSDKProtoRecordPB record : response.getResp().getCdcSdkProtoRecordsList()) { + try { + records.add(record); + } catch (Exception e) { + LOG.error("Exception caught while adding records, cannot add records further", e); + e.printStackTrace(); + fail(); + } + } + } + + /** + * This function is an overloaded function which creates a CDC stream on a table + * (in our case, test) in order to stream the data via that streamId.

+ * + * If no record format is specified then this takes the + * default record format as "proto" and if a record format is specified then + * this sets the record format to that one + * + * @throws Exception if unable to create a stream + */ + public void createStream() throws Exception { + setFormat("proto"); + createStreamUtil(""); + } + + /** + * Overloaded version of the createStream() with a parameter to specify the record format. + * @param recordFormat Format of the record to be streamed (json/proto) + * @throws Exception when unable to create a stream + */ + public void createStream(String recordFormat) throws Exception { + recordFormat = recordFormat.toLowerCase(Locale.ROOT); + if (!Objects.equals(recordFormat, "proto") && !Objects.equals(recordFormat, "json")) { + LOG.error("Invalid format specified, please specify one from JSON or PROTO"); + throw new RuntimeException("Invalid format specified, specify one from JSON or PROTO"); + } + + setFormat(recordFormat); + createStreamUtil(""); + } + + /** + * Overloaded version of the createStream() with a parameter to specify the record format + * and checkpoint type. + * @param recordFormat Format of the record to be streamed (json/proto) + * @param checkpointingType Type of checkpointing for the stream to be created (IMPLICIT/EXPLICIT) + * @throws Exception when unable to create a stream + */ + public void createStream(String recordFormat, String checkpointingType) throws Exception { + recordFormat = recordFormat.toLowerCase(Locale.ROOT); + checkpointingType = checkpointingType.toUpperCase(Locale.ROOT); + + if (!Objects.equals(recordFormat, "proto") && !Objects.equals(recordFormat, "json")) { + LOG.error("Invalid format specified, please specify one from JSON or PROTO"); + throw new RuntimeException("Invalid format specified, specify one from JSON or PROTO"); + } + + if(!Objects.equals(checkpointingType, "IMPLICIT") && + !Objects.equals(checkpointingType, "EXPLICIT")) { + LOG.error("Invalid checkpointing type specified"); + throw new RuntimeException("Invalid checkpointing type, " + + "specify one from IMPLICIT or EXPLICIT"); + } + + setFormat(recordFormat); + createStreamUtil(checkpointingType); + } + + /** + * Just a getter function to access the GetDBStreamInfoResponse after we have the DB Stream ID + * with us. + * + * @throws Exception when unable to access the DBStreamInfo + * @return GetDBStreamInfoResponse + * @see YBClient + */ + public GetDBStreamInfoResponse getDBStreamInfo() throws Exception { + if (syncClient == null) { + return null; + } + + return syncClient.getDBStreamInfo(dbStreamId); + } + + /** + * This function uses the YBClient and retrieves the current checkpoint for a particular table. + * + * @throws Exception when unable to access the checkpoint + * @return GetCheckpointResponse + * @see YBClient + * @see GetCheckpointResponse + * @see GetDBStreamInfoResponse + */ + public GetCheckpointResponse getCheckpoint() throws Exception { + if (syncClient == null) { + return null; + } + + GetDBStreamInfoResponse resp = syncClient.getDBStreamInfo(dbStreamId); + + String streamId = ""; + for (MasterReplicationOuterClass.GetCDCDBStreamInfoResponsePB.TableInfo tableInfo : + resp.getTableInfoList()) { + streamId = tableInfo.getStreamId().toStringUtf8(); + LOG.debug("Table StreamID: " + streamId); + } + if (streamId.isEmpty()) { + return null; + } + + LOG.debug("Tablet ID for getting checkpoint: " + tabletId); + + return syncClient.getCheckpoint(table, streamId, tabletId); + } + + /** + * This function is used to set the checkpoint in EXPLICIT mode using YBClient. + * + * @param term Checkpoint's term + * @param index Checkpoint's index + * @return SetCheckpointResponse + * @throws Exception when unable to set checkpoint + * @see YBClient + * @see SetCheckpointResponse + * @see GetDBStreamInfoResponse + * @see CdcService.TableInfo + */ + public SetCheckpointResponse setCheckpoint(long term, long index) throws Exception { + if (syncClient == null) { + LOG.info("Cannot set checkpoint, YBClient not initialized"); + return null; + } + + GetDBStreamInfoResponse resp = syncClient.getDBStreamInfo(dbStreamId); + String streamId = ""; + for (MasterReplicationOuterClass.GetCDCDBStreamInfoResponsePB.TableInfo tableInfo : + resp.getTableInfoList()) { + streamId = tableInfo.getStreamId().toStringUtf8(); + LOG.debug("Table StreamID: " + streamId); + } + if (streamId.isEmpty()) { + return null; + } + + return syncClient.commitCheckpoint(table, streamId, tabletId, term, index); + } + + /** + * This is used to create a stream to verify the snapshot feature of CDC.

+ * + * This is a separate function currently because: + *
    + *
  • The snapshot feature works for proto record only
  • + *
  • We need to set the checkpoint to a specific -1, -1 position to enable this
  • + *
+ * + * @param records This is a List to which the streamed records would be added + * @see org.yb.cdc.CdcService.CDCSDKProtoRecordPB + * @see List + */ + public void createStreamAndGetSnapshot(List records) throws Exception { + setFormat("proto"); + syncClient = createSyncClientForTest(); + tableId = getTableId(); + table = getTable(); + + if (dbStreamId == null || dbStreamId.isEmpty()) { + LOG.debug("Creating a new CDC DB stream"); + dbStreamId = syncClient.createCDCStream(table, + "yugabyte", FORMAT, "IMPLICIT").getStreamId(); + + LOG.debug(String.format("Created a new DB stream id: %s", dbStreamId)); + } else { + LOG.debug("Using an old cached stream id"); + } + + LOG.info("DB Stream id: " + dbStreamId); + + getSnapshotFromCDC(records); + } + + /** + * This function is used in conjunction with createStreamAndGetSnapshot(). It's a separate + * function because to test the snapshot feature, the checkpoint should be set at <-1, -1> + * + * @param records The streamed record/snapshot records would be added in this list + * @see GetChangesResponse + * @throws Exception if unable to get snapshots + */ + public void getSnapshotFromCDC(List records) throws Exception { + Checkpoint cp = new Checkpoint(-1, -1, "".getBytes(), -1, 0); + getResponseFromCDC(records, cp); + + // The first call would not return any values and will return a list without any records, + // to overcome this, call the getChanges function again. + if (records.isEmpty()) { + getResponseFromCDC(records, getSubscriberCheckpoint()); + } + } + + /** + * This function calls the getChangesCDCSDK() function in order to get the response from the CDC + * stream and adds the streamed record to the passed list. + * + * @param records List to store the CDC records + * @see GetChangesResponse + * @throws Exception if unable to get changes response from CDC + */ + public void getResponseFromCDC(List records) throws Exception { + Checkpoint cp = new Checkpoint(0, 0, new byte[]{}, 0, 0); + getResponseFromCDC(records, cp); + } + + public void getResponseFromCDC(List records, Checkpoint cp) throws Exception { + List locatedTablets = table.getTabletsLocations(30000); + List tabletIds = new ArrayList<>(); + + for (LocatedTablet tablet : locatedTablets) { + String tabletId = new String(tablet.getTabletId()); + LOG.debug(String.format("Polling for tablet: %s from checkpoint %s", tabletId, cp)); + this.tabletId = tabletId; // There is going to be a single tablet only. + tabletIds.add(tabletId); + } + + for (String tabletId : tabletIds) { + GetChangesResponse changesResponse = + syncClient.getChangesCDCSDK(table, dbStreamId, tabletId, + cp.getTerm(), cp.getIndex(), cp.getKey(), cp.getWriteId(), cp.getSnapshotTime()); + + if (FORMAT.equalsIgnoreCase("PROTO")) { + // Add records in proto. + addProtoRecords(changesResponse, records); + } + + updateCheckpointValues(changesResponse); + } + } + + /** + * Take the change response and update the cached checkpoint based on the values in the response. + * @param changesResponse The {@link GetChangesResponse} received after a + * {@link YBClient#getChangesCDCSDK} request. + * @see GetChangesResponse + * @see Checkpoint + */ + private void updateCheckpointValues(GetChangesResponse changesResponse) { + checkpoint = new Checkpoint(changesResponse.getTerm(), + changesResponse.getIndex(), + changesResponse.getKey(), + changesResponse.getWriteId(), + changesResponse.getSnapshotTime()); + } + + /** + * + * @return the cached checkpoint in the CDCSubscriber + * @throws NullPointerException + * @see Checkpoint + */ + public Checkpoint getSubscriberCheckpoint() throws NullPointerException { + if (checkpoint == null) { + throw new NullPointerException("Checkpoint has not been set yet"); + } + return checkpoint; + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriberClient.java b/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriberClient.java new file mode 100644 index 000000000000..b8d703863e6e --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/util/CDCSubscriberClient.java @@ -0,0 +1,42 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.util; + +import org.apache.log4j.Logger; +import org.yb.cdc.CdcService; +import org.yb.cdc.OutputClient; +import org.yb.client.YBTable; + +import java.util.ArrayList; +import java.util.List; + +public class CDCSubscriberClient implements OutputClient { + private static final Logger LOG = Logger.getLogger(CDCSubscriberClient.class); + + public CDCSubscriberClient(List records) { + this.records = records; + } + + public List getRecords() { + return records; + } + + private List records = new ArrayList<>(); + + @Override + public void applyChange(YBTable table, + CdcService.CDCSDKProtoRecordPB changeRecord) throws Exception { + records.add(changeRecord); + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/util/TestUtils.java b/java/yb-cdc/src/test/java/org/yb/cdc/util/TestUtils.java new file mode 100644 index 000000000000..6e72052af550 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/util/TestUtils.java @@ -0,0 +1,214 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.util; + +import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.ibatis.jdbc.ScriptRunner; +import org.apache.log4j.Logger; +import org.yb.cdc.CDCConsoleSubscriber; +import org.yb.cdc.CmdLineOpts; +import org.yb.cdc.ysql.TestBase; + +import java.io.*; +import java.net.URL; +import java.net.URLDecoder; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.sql.Connection; +import java.util.List; + +public class TestUtils { + private static final Logger LOG = Logger.getLogger(TestUtils.class); + + private static final Path resourcePath = Paths.get("src", "test", "resources"); + private static final String TEST_RESOURCES_PATH = resourcePath.toFile().getAbsolutePath(); + + private static String rootYBDirectory; + + public static CDCConsoleSubscriber initJavaClient(List outputList) throws Exception { + return runJavaClient(outputList, TEST_RESOURCES_PATH + "/config.properties"); + } + + private static CDCConsoleSubscriber runJavaClient(List outputList, String configFilePath) + throws Exception { + String[] args = {"-table_name", "yugabyte.test", "-config_file", configFilePath}; + + CmdLineOpts configuration = CmdLineOpts.createFromArgs(args); + CDCSubscriberClient testClient = new CDCSubscriberClient(outputList); + CDCConsoleSubscriber cdcObject = new CDCConsoleSubscriber(configuration, testClient); + + LOG.info("Starting CDCConsoleSubscriber..."); + + new Thread(cdcObject::run).start(); + + return cdcObject; + } + + /** + * This function clears the streamID in the passed dummy_file.properties file via command line. + * If no path is passed (as in case of tests), it would automatically go over and read the + * dummy_file.properties file in the resources directory of tests. + * + * @param customConfigPath The path to the properties file consisting of the required + * configuration settings. + * + */ + public static void cleanStreamIdOnly(String customConfigPath) { + try { + PropertiesConfiguration propConfig; + + if (customConfigPath == null || customConfigPath.isEmpty()) { + LOG.error("Provide a valid properties file path"); + System.exit(0); + } + + propConfig = new PropertiesConfiguration(customConfigPath); + propConfig.setProperty("stream.id", ""); + propConfig.save(); + } catch (ConfigurationException ce) { + ce.printStackTrace(); + } + } + + + public static void clearStreamId(String customConfigPath) { + cleanStreamIdOnly(customConfigPath); + } + + /** + * This function runs a SQL script using ScriptRunner on the specified connection. + * + * @param conn A Connection to connect to and execute the SQL script + * @param fileName File name of the SQL script with complete path in reference to the resources + * directory in the test directory of this project + * @see Connection + * @see ScriptRunner + */ + public static void runSqlScript(Connection conn, String fileName) { + LOG.info("Running the SQL script: " + fileName); + ScriptRunner sr = new ScriptRunner(conn); + sr.setAutoCommit(true); + + // This prevents ScriptWriter from printing the queries on terminal or logging them either. + sr.setLogWriter(null); + + final String sqlFile = TEST_RESOURCES_PATH + "/"+ fileName; + try { + Reader reader = new BufferedReader(new FileReader(sqlFile)); + sr.runScript(reader); + } catch (FileNotFoundException f) { + f.printStackTrace(); + } + } + + private static String urlToPath(URL u) { + try { + return URLDecoder.decode(u.getPath(), "UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + private static String findYbSrcRootContaining(String initialPath) { + File currentPath = new File(initialPath); + while (currentPath != null) { + if (new File(currentPath, "yb_build.sh").exists() && + new File(currentPath, "build-support").exists()) { + return currentPath.getAbsolutePath(); + } + currentPath = currentPath.getParentFile(); + } + return null; + } + + /** + * This function, when called, iterates over the parent directories and finds out the root + * directory "yugabyte-db" and returns the path to it. The path to the root directory is useful + * in executing yugabyted, yb-ctl, etc. + * + * @returns The absolute path to the root directory yugtabyte-db + * @throws RuntimeException When it is unable to find the root directory + */ + public static synchronized String findRootYBDirectory() throws Exception { + if (rootYBDirectory != null) { + return rootYBDirectory; + } + + final URL myUrl = TestBase.class.getProtectionDomain().getCodeSource().getLocation(); + final String pathToCode = urlToPath(myUrl); + final String currentDir = System.getProperty("user.dir"); + + // Try to find the YB directory root by navigating upward from either the source code location, + // or, if that does not work, from the current directory. + for (String initialPath : new String[] { pathToCode, currentDir }) { + // Cache the root dir so that we don't have to find it every time. + rootYBDirectory = findYbSrcRootContaining(initialPath); + if (rootYBDirectory != null) { + return rootYBDirectory; + } + } + throw new RuntimeException( + "Unable to find build dir! myUrl=" + myUrl + ", currentDir=" + currentDir); + } + + /** + * This function starts a single node cluster using yugabyted. The flow is that it initially + * calls destroyYugabyted() to destroy any existing cluster which could be running and then + * creates a new cluster itself. + */ + public static void startYugabyted() { + try { + LOG.info("Destroying yugabyted if running"); + destroyYugabyted(); + String ybRootDir = (rootYBDirectory == null) ? findRootYBDirectory() : rootYBDirectory; + Process process = Runtime.getRuntime().exec(ybRootDir + "/bin/yugabyted start"); + + int exitCode = process.waitFor(); + if (exitCode == 0) { + LOG.info("yugabyted started successfully"); + } else { + throw new Exception("yugabyted couldn't start, check if yb_build.sh is successful"); + } + + } catch (Exception e) { + LOG.error("Exception caught while starting yugabyted", e); + System.exit(0); + } + } + + /** + * This function destroys any existing yugabyted cluster (a cluster with a single node). + */ + public static void destroyYugabyted() { + try { + String ybRootDir = (rootYBDirectory == null) ? findRootYBDirectory() : rootYBDirectory; + Process process = Runtime.getRuntime().exec(ybRootDir + "/bin/yugabyted destroy"); + + int exitCode = process.waitFor(); + if (exitCode == 0) { + LOG.info("yugabyted destroyed successfully"); + } else { + if (exitCode == 1) { + LOG.error("yugabyted cluster not running"); + } else { + throw new Exception("yugabyted cluster couldn't be destroyed"); + } + } + } catch (Exception e) { + LOG.error("Exception caught while stopping yugabyted", e); + System.exit(0); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ycql/TestDatatypes.java b/java/yb-cdc/src/test/java/org/yb/cdc/ycql/TestDatatypes.java new file mode 100644 index 000000000000..eec8cefae6b6 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ycql/TestDatatypes.java @@ -0,0 +1,371 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ycql; + +import com.datastax.oss.driver.api.core.CqlSession; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +import org.junit.runner.RunWith; +import org.yb.cdc.CDCConsoleSubscriber; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.ExpectedRecordYCQLGeneric; +import org.yb.cdc.util.TestUtils; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.math.BigInteger; +import java.net.InetSocketAddress; +import java.sql.Timestamp; +import java.time.LocalTime; +import java.util.*; + +/** + * This test has been disabled for now since the YCQL support for CDC is not there yet. + * See the corresponding + * GitHub issue + */ +@Ignore("Disabled until fix for YCQL lands") +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestDatatypes { + private CqlSession session; + + public void assertRecords(ExpectedRecordYCQLGeneric[] expectedRecords) throws Exception{ + List outputList = new ArrayList<>(); + CDCConsoleSubscriber cdcSubscriberObj = TestUtils.initJavaClient(outputList); + + assertEquals(expectedRecords.length, outputList.size()); + + for (int i = 0; i < outputList.size(); ++i) { + ExpectedRecordYCQLGeneric.checkRecord(outputList.get(i), expectedRecords[i]); + } + + cdcSubscriberObj.close(); + } + + @Before + public void setUp() { + TestUtils.clearStreamId(""); + String createKeyspace = "create keyspace if not exists yugabyte;"; + String dropTable = "drop table if exists yugabyte.test;"; + + session = CqlSession + .builder() + .addContactPoint(new InetSocketAddress("127.0.0.1", 9042)) + .withLocalDatacenter("datacenter1") + .build(); + + session.execute(createKeyspace); + session.execute(dropTable); + } + + @Test + public void testConnection() { + assertFalse(session.isClosed()); + } + + @Test + public void testYCQLInet() throws Exception { + + session.execute("create table yugabyte.test (a int primary key, b inet);"); + + session.execute("insert into yugabyte.test (a, b) values (1, '127.0.0.1');"); + + int arraySize = 3; + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, "127.0.0.1", Op.INSERT), + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testUUID() throws Exception { + // Works the same for timeuuid. + session.execute("create table yugabyte.test (a int primary key, b uuid);"); + session.execute( + "insert into yugabyte.test (a, b) values (1, 123e4567-e89b-12d3-a456-426655440000);"); + session.execute( + "insert into yugabyte.test (a, b) values (2, 123e4567-e89b-12d3-a456-426655440000);"); + session.execute( + "insert into yugabyte.test (a, b) values (3, 123e4567-e89b-12d3-a456-426655440000);"); + + // UUID.randomUUID() is just being used as a placeholder + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, UUID.randomUUID(), Op.DDL), + new ExpectedRecordYCQLGeneric<>( + 1, + UUID.fromString("123e4567-e89b-12d3-a456-426655440000"), + Op.INSERT), + new ExpectedRecordYCQLGeneric<>( + 2, + UUID.fromString("123e4567-e89b-12d3-a456-426655440000"), + Op.INSERT), + new ExpectedRecordYCQLGeneric<>( + 3, + UUID.fromString("123e4567-e89b-12d3-a456-426655440000"), + Op.INSERT), + new ExpectedRecordYCQLGeneric<>(0, UUID.randomUUID(), Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testYCQLJsonb() throws Exception { + session.execute("create table yugabyte.test (a int primary key, b jsonb);"); + session.execute("insert into yugabyte.test (a, b) values (1, '{\"fName\":\"vaibhav\"}');"); + session.execute("insert into yugabyte.test (a, b) values (2, '{\"lName\":\"kushwaha\"}');"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL), + new ExpectedRecordYCQLGeneric<>( + 1, + "{\"fName\":\"vaibhav\"}", + Op.INSERT), + new ExpectedRecordYCQLGeneric<>( + 2, + "{\"lName\":\"kushwaha\"}", + Op.INSERT), + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testTimestamp() throws Exception { + Timestamp ts1 = new Timestamp(System.currentTimeMillis()); + String s1 = ts1.toString(); + String s2 = ts1.toString(); + + session.execute("create table yugabyte.test (a int primary key, b timestamp);"); + + session.execute("insert into yugabyte.test (a, b) values (1, '" + s1 + "');"); + session.execute("insert into yugabyte.test (a, b) values (2, '" + s2 + "');"); + session.execute("delete from yugabyte.test where a = 2;"); + + Timestamp ts2 = new Timestamp(System.currentTimeMillis()); + String s3 = ts2.toString(); + session.execute("insert into yugabyte.test (a, b) values (3, '" + s3 + "');"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, new Date(), Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, new Date(ts1.getTime()), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, new Date(ts1.getTime()), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, new Date(), Op.DELETE), + new ExpectedRecordYCQLGeneric<>(3, new Date(ts2.getTime()), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(0, new Date(), Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testInteger() throws Exception { + // Works with bigint, counter, int, integer, smallint, tinyint + + session.execute("create table yugabyte.test (a int primary key, b tinyint) " + + "with transactions = {'enabled' : true};"); + session.execute("insert into yugabyte.test (a, b) values (1, 2);"); + session.execute("insert into yugabyte.test (a, b) values (2, 3);"); + session.execute("insert into yugabyte.test (a, b) values (4, 5);"); + session.execute("insert into yugabyte.test (a, b) values (6, 7);"); + session.execute("begin transaction delete from yugabyte.test where a = 2; end transaction;"); + + session.execute("update yugabyte.test set b = 6 where a = 4;"); + session.execute("delete from yugabyte.test where a = 6;"); + + /* + * If you are testing for bigint, make sure you pass the value in expected record as Long + * i.e. ExpectedRecordGeneric<>(int-value, 2L, Op.SomeOperation) + */ + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, 0, Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, 2, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, 3, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(4, 5, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(6, 7, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, 0, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, 0, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(4, 6, Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(6, 0, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, 0, Op.DDL), + }; + + assertRecords(expectedRecords); + } + + @Test + public void testBoolean() throws Exception { + + session.execute("create table yugabyte.test (a int primary key, b boolean) " + + "with transactions = {'enabled' : true};"); + session.execute("insert into yugabyte.test (a, b) values (1, FALSE);"); + session.execute("insert into yugabyte.test (a, b) values (2, TRUE);"); + session.execute("insert into yugabyte.test (a, b) values (4, TRUE);"); + session.execute("insert into yugabyte.test (a, b) values (8, FALSE);"); + session.execute("begin transaction delete from yugabyte.test where a = 2; end transaction;"); + + String multiQueryTxn = "begin transaction " + + "insert into yugabyte.test (a, b) values (100, FALSE); " + + "delete from yugabyte.test where a = 1; " + + "end transaction;"; + + session.execute(multiQueryTxn); + + session.execute("update yugabyte.test set b = TRUE where a = 100;"); + session.execute("update yugabyte.test set b = FALSE where a = 4;"); + session.execute("delete from yugabyte.test where a = 8;"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, false, Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, false, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, true, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(4, true, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(8, false, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, false, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, false, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(100, false, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(1, false, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, false, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(100, true, Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(4, false, Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(8, false, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, false, Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testDouble() throws Exception { + // Works the same for FLOAT. + + session.execute("create table yugabyte.test (a int primary key, b double) " + + "with transactions = {'enabled' : true};"); + session.execute("insert into yugabyte.test (a, b) values (1, 32.123);"); + session.execute("insert into yugabyte.test (a, b) values (2, 12.345);"); + session.execute("insert into yugabyte.test (a, b) values (4, 98.765);"); + session.execute("begin transaction delete from yugabyte.test where a = 2; end transaction;"); + + session.execute( + "begin transaction update yugabyte.test set b = 11.223 where a = 1; end transaction;"); + session.execute("delete from yugabyte.test where a = 4;"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, 0, Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, 32.123, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, 12.345, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(4, 98.765, Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, 0, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, 0, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(1, 11.223, Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(0, 0, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(4, 0, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, 0, Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testTime() throws Exception { + session.execute("create table yugabyte.test (a int primary key, b time) " + + "with transactions = {'enabled' : true};"); + + session.execute("insert into yugabyte.test (a, b) values (1, '11:00:00');"); + session.execute("insert into yugabyte.test (a, b) values (2, '21:00:00');"); + + String multiQueryTxn = "begin transaction " + + "delete from yugabyte.test where a = 1; " + + "insert into yugabyte.test (a, b) values (3, '17:30:25'); " + + "end transaction;"; + session.execute(multiQueryTxn); + + // LocalTime.MIN is a random value used as placeholder. + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, LocalTime.MIN, Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, LocalTime.of(11, 0, 0), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, LocalTime.of(21, 0, 0), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(1, LocalTime.MIN, Op.DELETE), + new ExpectedRecordYCQLGeneric<>(3, LocalTime.of(17, 30, 25), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(0, LocalTime.MIN, Op.COMMIT), + new ExpectedRecordYCQLGeneric<>(0, LocalTime.MIN, Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testVarint() throws Exception { + session.execute("create table yugabyte.test (a int primary key, b varint) " + + "with transactions = {'enabled':true};"); + + session.execute("insert into yugabyte.test (a, b) values (1, 1001);"); + session.execute("insert into yugabyte.test (a, b) values (2, 2001);"); + + /* + * This would stream a DELETE record across CDC even though the record is not present in + * the table, this behaviour is because YCQL uses upsert semantics where if a row is not + * present while performing an UPDATE or DELETE operation, it would be inserted first and + * then the operation would be performed. + * + * DO NOTE THAT even though the row is inserted first, it won't come + * in the CDC streamed records. + */ + session.execute("delete from yugabyte.test where a = 33;"); + + session.execute("update yugabyte.test set b = 10000 where a = 2;"); + session.execute("update yugabyte.test set b = 20000 where a = 1;"); + + session.execute("delete from yugabyte.test where a = 1;"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, BigInteger.valueOf(0), Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, BigInteger.valueOf(1001), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, BigInteger.valueOf(2001), Op.INSERT), + new ExpectedRecordYCQLGeneric<>(33, BigInteger.valueOf(0), Op.DELETE), + new ExpectedRecordYCQLGeneric<>(2, BigInteger.valueOf(10000), Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(1, BigInteger.valueOf(20000), Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(1, BigInteger.valueOf(0), Op.DELETE), + new ExpectedRecordYCQLGeneric<>(0, BigInteger.valueOf(0), Op.DDL) + }; + + assertRecords(expectedRecords); + } + + @Test + public void testBlob() throws Exception { + session.execute("create table yugabyte.test (a int primary key, b blob);"); + + session.execute("insert into yugabyte.test (a, b) values (1, 0xff);"); + session.execute("insert into yugabyte.test (a, b) values (2, 0x10ef);"); + + ExpectedRecordYCQLGeneric[] expectedRecords = new ExpectedRecordYCQLGeneric[] { + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL), + new ExpectedRecordYCQLGeneric<>(1, "ff", Op.INSERT), + new ExpectedRecordYCQLGeneric<>(2, "10ef", Op.INSERT), + new ExpectedRecordYCQLGeneric<>(1, "01", Op.UPDATE), + new ExpectedRecordYCQLGeneric<>(0, "", Op.DDL) + }; + + assertRecords(expectedRecords); + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/Test3IntCol.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/Test3IntCol.java new file mode 100644 index 000000000000..2c14bc941f1a --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/Test3IntCol.java @@ -0,0 +1,230 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; + +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.common.ExpectedRecord3Proto; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.cdc.util.TestUtils; + +import java.util.ArrayList; +import java.util.List; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Test; +import org.yb.util.YBTestRunnerNonTsanOnly; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class Test3IntCol extends CDCBaseClass { + private Logger LOG = Logger.getLogger(Test3IntCol.class); + + private void executeScriptAssertRecords(ExpectedRecord3Proto[] expectedRecords, + String sqlScript) throws Exception { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + if (!sqlScript.isEmpty()) { + TestUtils.runSqlScript(connection, sqlScript); + } else { + LOG.info("No SQL script specified..."); + } + + List outputList = new ArrayList<>(); + setServerFlag(getTserverHostAndPort(), "cdc_max_stream_intent_records", "25"); + testSubscriber.getResponseFromCDC(outputList); + + int expRecordIndex = 0; + int processedRecords = 0; + for (int i = 0; i < outputList.size(); ++i) { + // Ignoring the DDLs if any. + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + ExpectedRecord3Proto.checkRecord(outputList.get(i), + new ExpectedRecord3Proto(-1, -1, -1, Op.DDL)); + continue; + } + + ExpectedRecord3Proto.checkRecord(outputList.get(i), expectedRecords[expRecordIndex++]); + ++processedRecords; + } + + // NOTE: processedRecords will be the same as expRecordIndex. + assertEquals(expectedRecords.length, processedRecords); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c int);"); + } + + // Insert rows in batch outside an explicit transaction. + // Expected records: 5 (4_INSERT, WRITE) + // WRITE op is generated as this batch insert would be treated as multi shard transaction. + @Test + public void testInsertionInBatchSingleShard() { + try { + ExpectedRecord3Proto[] expectedRecords = { + new ExpectedRecord3Proto(7, 8, 9, Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, Op.BEGIN), + new ExpectedRecord3Proto(4, 5, 6, Op.INSERT), + new ExpectedRecord3Proto(34, 35, 45, Op.INSERT), + new ExpectedRecord3Proto(1000, 1001, 1004, Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, Op.COMMIT) + }; + + executeScriptAssertRecords(expectedRecords, "cdc_insert_in_batch_outside_txn.sql"); + } catch (Exception e) { + LOG.error("Test to insert in batch outside BTET block failed with exception: ", e); + fail(); + } + } + + // Begin transaction, insert rows in batch, commit. + // Expected records: 5 (4_INSERT, WRITE) + @Test + public void testInsertionInBatch() { + try { + // Expect 5 records, 4 INSERT + 1 WRITE + ExpectedRecord3Proto[] expectedRecords = { + new ExpectedRecord3Proto(-1, -1, -1, Op.BEGIN), + new ExpectedRecord3Proto(7, 8, 9, Op.INSERT), + new ExpectedRecord3Proto(4, 5, 6, Op.INSERT), + new ExpectedRecord3Proto(34, 35, 45, Op.INSERT), + new ExpectedRecord3Proto(1000, 1001, 1004, Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, Op.COMMIT) + }; + + executeScriptAssertRecords(expectedRecords, "cdc_insert_in_batch.sql"); + } catch (Exception e) { + LOG.error("Test to insert in batch failed with exception: ", e); + fail(); + } + } + + // Begin transaction, perform multiple queries, rollback. + // Expected records: 0 + @Test + public void testRollbackTransaction() throws Exception { + try { + ExpectedRecord3Proto[] expectedRecords = { + }; + + executeScriptAssertRecords(expectedRecords, "cdc_long_txn_rollback.sql"); + } catch (Exception e) { + LOG.error("Test to rollback a long transaction failed with exception: ", e); + fail(); + } + } + + // Execute long script containing multiple operations of all kinds. + // Expected records: 50 (see script for more details) + @Test + public void testLongRunningScript() { + try { + ExpectedRecord3Proto[] expectedRecords = { + new ExpectedRecord3Proto(1, 2, 3, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(1, 2, 4, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(1, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(2, 2, 4, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(7, 8, 9, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(7, 17, 9, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(6, 7, 8, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(6, 7, 17, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(6, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(15, 7, 17, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(11, 12, 13, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(11, 13, 13, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(11, 13, 14, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(12, 112, 113, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(12, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(13, 113, 114, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(13, 113, 115, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(13, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(14, 113, 115, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(17, 114, 115, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(17, 114, 116, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(17, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(18, 114, 116, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(18, 115, 116, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(18, 115, 117, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(18, 116, 117, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(18, 116, 118, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(20, 21, 22, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(20, 22, 22, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(20, 22, 23, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(20, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(21, 22, 23, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(21, 23, 23, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(21, 23, 24, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(21, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(21, 23, 24, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(-1, -2, -3, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-4, -5, -6, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-11, -12, -13, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(404, 405, 406, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(104, 204, 304, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(41, 43, 44, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(41, 44, 44, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(41, 44, 45, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT), + new ExpectedRecord3Proto(41, 0, 0, CdcService.RowMessage.Op.DELETE), + new ExpectedRecord3Proto(41, 43, 44, CdcService.RowMessage.Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.BEGIN), + new ExpectedRecord3Proto(41, 44, 44, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(41, 44, 45, CdcService.RowMessage.Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, CdcService.RowMessage.Op.COMMIT) + }; + + executeScriptAssertRecords(expectedRecords, "cdc_long_script.sql"); + + } catch (Exception e) { + LOG.error("Test to verify CDC behaviour with proto records failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestAllDatatypes.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestAllDatatypes.java new file mode 100644 index 000000000000..c2b6044a080d --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestAllDatatypes.java @@ -0,0 +1,566 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.common.ExpectedRecordYSQL; +import org.yb.cdc.common.HelperValues; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.cdc.util.TestUtils; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Test; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestAllDatatypes extends CDCBaseClass { + private final Logger LOG = Logger.getLogger(TestAllDatatypes.class); + + public void assertRecordsOnly(ExpectedRecordYSQL[] expectedRecords, + CDCSubscriber testSubscriber) throws Exception { + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + int expRecordIndex = 0; + int processedRecords = 0; + for (int i = 0; i < outputList.size(); ++i) { + // Ignoring the DDLs. + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + ExpectedRecordYSQL.checkRecord(outputList.get(i), + new ExpectedRecordYSQL<>(-1, "", Op.DDL)); + continue; + } + + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[expRecordIndex++]); + ++processedRecords; + } + // processedRecords will be the same as expRecordIndex + assertEquals(expectedRecords.length, processedRecords); + } + + public void createTables() throws Exception { + statement.execute("create table testbit (a int primary key, b bit(6));"); + statement.execute("create table testboolean (a int primary key, b boolean);"); + statement.execute("create table testbox (a int primary key, b box);"); + statement.execute("create table testbytea (a int primary key, b bytea);"); + statement.execute("create table testcidr (a int primary key, b cidr);"); + statement.execute("create table testcircle (a int primary key, b circle);"); + statement.execute("create table testdate (a int primary key, b date);"); + statement.execute("create table testdouble (a int primary key, b double precision);"); + statement.execute("create table testinet (a int primary key, b inet);"); + statement.execute("create table testint (a int primary key, b int);"); + statement.execute("create table testjson (a int primary key, b json);"); + statement.execute("create table testjsonb (a int primary key, b jsonb);"); + statement.execute("create table testline (a int primary key, b line);"); + statement.execute("create table testlseg (a int primary key, b lseg);"); + statement.execute("create table testmacaddr8 (a int primary key, b macaddr8);"); + statement.execute("create table testmacaddr (a int primary key, b macaddr);"); + statement.execute("create table testmoney (a int primary key, b money);"); + statement.execute("create table testnumeric (a int primary key, b numeric);"); + statement.execute("create table testpath (a int primary key, b path);"); + statement.execute("create table testpoint (a int primary key, b point);"); + statement.execute("create table testpolygon (a int primary key, b polygon);"); + statement.execute("create table testtext (a int primary key, b text);"); + statement.execute("create table testtime (a int primary key, b time);"); + statement.execute("create table testtimestamp (a int primary key, b timestamp);"); + statement.execute("create table testtimetz (a int primary key, b timetz);"); + statement.execute("create table testuuid (a int primary key, b uuid);"); + statement.execute("create table testvarbit (a int primary key, b varbit(10));"); + statement.execute("create table testtstz (a int primary key, b timestamptz);"); + statement.execute("create table testint4range (a int primary key, b int4range);"); + statement.execute("create table testint8range (a int primary key, b int8range);"); + statement.execute("create table testtsrange (a int primary key, b tsrange);"); + statement.execute("create table testtstzrange (a int primary key, b tstzrange);"); + statement.execute("create table testdaterange (a int primary key, b daterange);"); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + } + + @Test + public void testAllTypes() { + try { + statement.execute(HelperValues.dropAllTablesWithTypes); + createTables(); + + CDCSubscriber bitSub = new CDCSubscriber("testbit", getMasterAddresses()); + bitSub.createStream("proto"); + CDCSubscriber booleanSub = new CDCSubscriber("testboolean", getMasterAddresses()); + booleanSub.createStream("proto"); + CDCSubscriber boxSub = new CDCSubscriber("testbox", getMasterAddresses()); + boxSub.createStream("proto"); + CDCSubscriber byteaSub = new CDCSubscriber("testbytea", getMasterAddresses()); + byteaSub.createStream("proto"); + CDCSubscriber cidrSub = new CDCSubscriber("testcidr", getMasterAddresses()); + cidrSub.createStream("proto"); + CDCSubscriber circleSub = new CDCSubscriber("testcircle", getMasterAddresses()); + circleSub.createStream("proto"); + CDCSubscriber dateSub = new CDCSubscriber("testdate", getMasterAddresses()); + dateSub.createStream("proto"); + CDCSubscriber doubleSub = new CDCSubscriber("testdouble", getMasterAddresses()); + doubleSub.createStream("proto"); + CDCSubscriber inetSub = new CDCSubscriber("testinet", getMasterAddresses()); + inetSub.createStream("proto"); + CDCSubscriber intSub = new CDCSubscriber("testint", getMasterAddresses()); + intSub.createStream("proto"); + CDCSubscriber jsonSub = new CDCSubscriber("testjson", getMasterAddresses()); + jsonSub.createStream("proto"); + CDCSubscriber jsonbSub = new CDCSubscriber("testjsonb", getMasterAddresses()); + jsonbSub.createStream("proto"); + CDCSubscriber lineSub = new CDCSubscriber("testline", getMasterAddresses()); + lineSub.createStream("proto"); + CDCSubscriber lsegSub = new CDCSubscriber("testlseg", getMasterAddresses()); + lsegSub.createStream("proto"); + CDCSubscriber macaddr8Sub = new CDCSubscriber("testmacaddr8", getMasterAddresses()); + macaddr8Sub.createStream("proto"); + CDCSubscriber macaddrSub = new CDCSubscriber("testmacaddr", getMasterAddresses()); + macaddrSub.createStream("proto"); + CDCSubscriber moneySub = new CDCSubscriber("testmoney", getMasterAddresses()); + moneySub.createStream("proto"); + CDCSubscriber numericSub = new CDCSubscriber("testnumeric", getMasterAddresses()); + numericSub.createStream("proto"); + CDCSubscriber pathSub = new CDCSubscriber("testpath", getMasterAddresses()); + pathSub.createStream("proto"); + CDCSubscriber pointSub = new CDCSubscriber("testpoint", getMasterAddresses()); + pointSub.createStream("proto"); + CDCSubscriber polygonSub = new CDCSubscriber("testpolygon", getMasterAddresses()); + polygonSub.createStream("proto"); + CDCSubscriber textSub = new CDCSubscriber("testtext", getMasterAddresses()); + textSub.createStream("proto"); + CDCSubscriber timeSub = new CDCSubscriber("testtime", getMasterAddresses()); + timeSub.createStream("proto"); + CDCSubscriber timestampSub = new CDCSubscriber("testtimestamp", getMasterAddresses()); + timestampSub.createStream("proto"); + CDCSubscriber timetzSub = new CDCSubscriber("testtimetz", getMasterAddresses()); + timetzSub.createStream("proto"); + CDCSubscriber uuidSub = new CDCSubscriber("testuuid", getMasterAddresses()); + uuidSub.createStream("proto"); + CDCSubscriber varbitSub = new CDCSubscriber("testvarbit", getMasterAddresses()); + varbitSub.createStream("proto"); + CDCSubscriber tstzSub = new CDCSubscriber("testtstz", getMasterAddresses()); + tstzSub.createStream("proto"); + CDCSubscriber int4rangeSub = new CDCSubscriber("testint4range", getMasterAddresses()); + int4rangeSub.createStream("proto"); + CDCSubscriber int8rangeSub = new CDCSubscriber("testint8range", getMasterAddresses()); + int8rangeSub.createStream("proto"); + CDCSubscriber tsrangeSub = new CDCSubscriber("testtsrange", getMasterAddresses()); + tsrangeSub.createStream("proto"); + CDCSubscriber tstzrangeSub = new CDCSubscriber("testtstzrange", getMasterAddresses()); + tstzrangeSub.createStream("proto"); + CDCSubscriber daterangeSub = new CDCSubscriber("testdaterange", getMasterAddresses()); + daterangeSub.createStream("proto"); + + TestUtils.runSqlScript(connection, "sql_datatype_script/complete_datatype_test.sql"); + + // -1 is used as a placeholder only. + ExpectedRecordYSQL[] expectedRecordsInteger = new ExpectedRecordYSQL[]{ + new ExpectedRecordYSQL<>(1, 2, Op.INSERT), + new ExpectedRecordYSQL<>(3, 4, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, 5, Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(7, 8, Op.INSERT), + new ExpectedRecordYSQL<>(7, "", Op.DELETE), + new ExpectedRecordYSQL<>(8, 8, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(8, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsInteger, intSub); + + ExpectedRecordYSQL[] expectedRecordsBoolean = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, false, Op.INSERT), + new ExpectedRecordYSQL<>(3, true, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, false, Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(1, false, Op.DELETE) + }; + assertRecordsOnly(expectedRecordsBoolean, booleanSub); + + ExpectedRecordYSQL[] expectedRecordsDouble = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, 10.42, Op.INSERT), + new ExpectedRecordYSQL<>(3, 0.5, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "", Op.DELETE), + new ExpectedRecordYSQL<>(4, 0.5, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsDouble, doubleSub); + + ExpectedRecordYSQL[] expectedRecordsText = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "sample string with pk 1", Op.INSERT), + new ExpectedRecordYSQL<>(3, "sample string with pk 3", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(2, "sample string with pk 2", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(3, "random sample string", Op.UPDATE) + }; + assertRecordsOnly(expectedRecordsText, textSub); + + ExpectedRecordYSQL[] expectedRecordsUuid = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "ffffffff-ffff-ffff-ffff-ffffffffffff", Op.INSERT), + new ExpectedRecordYSQL<>(3, "ffffffff-ffff-ffff-ffff-ffffffffffff", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "123e4567-e89b-12d3-a456-426655440000", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(2, "123e4567-e89b-12d3-a456-426655440000", Op.INSERT) + }; + assertRecordsOnly(expectedRecordsUuid, uuidSub); + + ExpectedRecordYSQL[] expectedRecordsTimestamp = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "2017-07-04 12:30:30", Op.INSERT), + new ExpectedRecordYSQL<>(2, "2021-09-29 00:00:00", Op.INSERT), + new ExpectedRecordYSQL<>(1, "1970-01-01 00:00:10", Op.UPDATE) + }; + assertRecordsOnly(expectedRecordsTimestamp, timestampSub); + + ExpectedRecordYSQL[] expectedRecordsDate = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "2021-09-20", Op.INSERT), + new ExpectedRecordYSQL<>(1, "2021-09-29", Op.UPDATE), + new ExpectedRecordYSQL<>(2, "2000-01-01", Op.INSERT), + new ExpectedRecordYSQL<>(2, "", Op.DELETE), + new ExpectedRecordYSQL<>(3, "1970-01-01", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "", Op.DELETE), + new ExpectedRecordYSQL<>(4, "1970-01-01", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsDate, dateSub); + + ExpectedRecordYSQL[] expectedRecordsInet = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "127.0.0.1", Op.INSERT), + new ExpectedRecordYSQL<>(2, "0.0.0.0", Op.INSERT), + new ExpectedRecordYSQL<>(3, "192.168.1.1", Op.INSERT), + new ExpectedRecordYSQL<>(3, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsInet, inetSub); + + ExpectedRecordYSQL[] expectedRecordsMacaddr = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "2c:54:91:88:c9:e3", Op.INSERT), + new ExpectedRecordYSQL<>(1, "2c:54:91:e8:99:d2", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "2c:54:91:e8:99:d2", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsMacaddr, macaddrSub); + + ExpectedRecordYSQL[] expectedRecordsMacaddr8 = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "22:00:5c:03:55:08:01:02", Op.INSERT), + new ExpectedRecordYSQL<>(1, "22:00:5c:04:55:08:01:02", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "22:00:5c:03:55:08:01:02", Op.INSERT), + new ExpectedRecordYSQL<>(2, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(3, "22:00:5c:05:55:08:01:02", Op.INSERT), + new ExpectedRecordYSQL<>(3, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsMacaddr8, macaddr8Sub); + + ExpectedRecordYSQL[] expectedRecordsJson = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "{\"first_name\":\"vaibhav\"}", Op.INSERT), + new ExpectedRecordYSQL<>(2, "{\"last_name\":\"kushwaha\"}", Op.INSERT), + new ExpectedRecordYSQL<>(2, "{\"name\":\"vaibhav kushwaha\"}", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(3, "{\"a\":97, \"b\":\"98\"}", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsJson, jsonSub); + + // Do note that there is a space after the colon (:) coming into the streamed records. + ExpectedRecordYSQL[] expectedRecordsJsonb = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "{\"first_name\": \"vaibhav\"}", Op.INSERT), + new ExpectedRecordYSQL<>(2, "{\"last_name\": \"kushwaha\"}", Op.INSERT), + new ExpectedRecordYSQL<>(2, "{\"name\": \"vaibhav kushwaha\"}", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(3, "{\"a\": 97, \"b\": \"98\"}", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsJsonb, jsonbSub); + + ExpectedRecordYSQL[] expectedRecordsBit = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "001111", Op.INSERT), + new ExpectedRecordYSQL<>(2, "110101", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "111111", Op.INSERT), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(0, "000000", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(2, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsBit, bitSub); + + ExpectedRecordYSQL[] expectedRecordsVarbit = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "001111", Op.INSERT), + new ExpectedRecordYSQL<>(2, "1101011101", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "11", Op.INSERT), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(0, "0", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(2, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsVarbit, varbitSub); + + ExpectedRecordYSQL[] expectedRecordsTime = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "11:30:59", Op.INSERT), + new ExpectedRecordYSQL<>(1, "23:30:59", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "00:00:01", Op.INSERT), + new ExpectedRecordYSQL<>(2, "00:01:00", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(2, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsTime, timeSub); + + ExpectedRecordYSQL[] expectedRecordsTimetz = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "11:30:59+05:30", Op.INSERT), + new ExpectedRecordYSQL<>(1, "23:30:59+05:30", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "00:00:01+00", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(2, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsTimetz, timetzSub); + + ExpectedRecordYSQL[] expectedRecordsNumeric = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, 20.5, Op.INSERT), + new ExpectedRecordYSQL<>(2, 100.75, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, 3.456, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + }; + assertRecordsOnly(expectedRecordsNumeric, numericSub); + + ExpectedRecordYSQL[] expectedRecordsMoney = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "$100.50", Op.INSERT), + new ExpectedRecordYSQL<>(2, "$10.12", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(3, "$1.23", Op.INSERT), + new ExpectedRecordYSQL<>(1, "$90.50", Op.UPDATE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT), + new ExpectedRecordYSQL<>(2, "", Op.DELETE) + }; + assertRecordsOnly(expectedRecordsMoney, moneySub); + + ExpectedRecordYSQL[] expectedRecordsCidr = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "10.1.0.0/16", Op.INSERT), + new ExpectedRecordYSQL<>(1, "12.2.0.0/22", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "12.2.0.0/22", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsCidr, cidrSub); + + ExpectedRecordYSQL[] expectedRecordsBytea = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "\\x01", Op.INSERT), + new ExpectedRecordYSQL<>(1, "\\xdeadbeef", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "\\xdeadbeef", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsBytea, byteaSub); + + ExpectedRecordYSQL[] expectedRecordsBox = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "(8,9),(1,3)", Op.INSERT), + new ExpectedRecordYSQL<>(1, "(10,31),(8,9)", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(2, "(10,31),(8,9)", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsBox, boxSub); + + ExpectedRecordYSQL[] expectedRecordsCircle = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(10, "<(2,3),32>", Op.INSERT), + new ExpectedRecordYSQL<>(10, "<(0,0),10>", Op.UPDATE), + new ExpectedRecordYSQL<>(10, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1000, "<(0,0),4>", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsCircle, circleSub); + + ExpectedRecordYSQL[] expectedRecordsPath = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(23, "((1,2),(20,-10))", Op.INSERT), + new ExpectedRecordYSQL<>(23, "((-1,-1))", Op.UPDATE), + new ExpectedRecordYSQL<>(23, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(34, "((0,0),(3,4),(5,5),(1,2))", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsPath, pathSub); + + ExpectedRecordYSQL[] expectedRecordsPoint = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(11, "(0,-1)", Op.INSERT), + new ExpectedRecordYSQL<>(11, "(1,3)", Op.UPDATE), + new ExpectedRecordYSQL<>(11, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(21, "(33,44)", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsPoint, pointSub); + + ExpectedRecordYSQL[] expectedRecordsPolygon = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "((1,3),(4,12),(2,4))", Op.INSERT), + new ExpectedRecordYSQL<>(1, "((1,3),(4,12),(2,4),(1,4))", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(27, "((1,3),(2,4),(1,4))", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsPolygon, polygonSub); + + ExpectedRecordYSQL[] expectedRecordsLine = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "{1,2,-8}", Op.INSERT), + new ExpectedRecordYSQL<>(1, "{1,1,-5}", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(29, "{2.5,-1,0}", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsLine, lineSub); + + ExpectedRecordYSQL[] expectedRecordsLseg = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "[(0,0),(2,4)]", Op.INSERT), + new ExpectedRecordYSQL<>(1, "[(-1,-1),(10,-8)]", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(37, "[(1,3),(3,5)]", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsLseg, lsegSub); + + ExpectedRecordYSQL[] expectedRecordsTimestamptz = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "1969-12-31 18:40:00+00", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "2021-12-31 18:40:00+00", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsTimestamptz, tstzSub); + + ExpectedRecordYSQL[] expectedRecordsInt4Range = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "[5,14)", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "[6,43)", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsInt4Range, int4rangeSub); + + ExpectedRecordYSQL[] expectedRecordsInt8Range = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "[5,15)", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "[2,100000)", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsInt8Range, int8rangeSub); + + ExpectedRecordYSQL[] expectedRecordsTsRange = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "(\"1970-01-01 00:00:00\",\"2000-01-01 12:00:00\")", + Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "(\"1970-01-01 00:00:00\",\"2022-11-01 12:00:00\")", + Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsTsRange, tsrangeSub); + + ExpectedRecordYSQL[] expectedRecordsTstzRange = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, + "(\"2017-07-04 12:30:30+00\",\"2021-07-04 07:00:30+00\")", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, + "(\"2017-07-04 12:30:30+00\",\"2021-10-04 07:00:30+00\")", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsTstzRange, tstzrangeSub); + + ExpectedRecordYSQL[] expectedRecordsDateRange = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, "[2019-10-08,2021-10-07)", Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, "[2019-10-08,2020-10-07)", Op.UPDATE), + new ExpectedRecordYSQL<>(1, "", Op.DELETE), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + assertRecordsOnly(expectedRecordsDateRange, daterangeSub); + } catch (Exception e) { + LOG.error("Failed to test all datatypes", e); + fail(); + } + } + + @Test + public void testDefaultForAllTypes() { + try { + assertFalse(statement.execute("drop table if exists testdefault;")); + assertFalse(statement.execute(HelperValues.createTableWithDefaults)); + + CDCSubscriber testSubscriber = new CDCSubscriber("testdefault", getMasterAddresses()); + testSubscriber.createStream("proto"); + + assertEquals(1, statement.executeUpdate("insert into testdefault values (1);")); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + boolean checkedInsertRecord = false; + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.INSERT) { + checkedInsertRecord = true; + + int changesCount = outputList.get(i).getRowMessage().getNewTupleCount(); + // there are 34 columns including the primary key one + assertEquals(34, changesCount); + assertEquals(1, outputList.get(i).getRowMessage().getNewTuple(0).getDatumInt32()); + + for (int j = 1; j < changesCount; ++j) { + Object streamedValue = ExpectedRecordYSQL.getValue(outputList.get(i), j); + assertEquals(HelperValues.expectedDefaultValues[j-1], streamedValue); + } + } + } + + // Just check whether an INSERT record was streamed. + assertTrue(checkedInsertRecord); + } catch (Exception e) { + LOG.error("Test to verify default value streaming for all types failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestArrayTypes.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestArrayTypes.java new file mode 100644 index 000000000000..f67bd4832163 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestArrayTypes.java @@ -0,0 +1,564 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.common.ExpectedRecordYSQL; +import org.yb.cdc.common.HelperValues; +import org.yb.cdc.util.CDCSubscriber; + +import org.yb.cdc.util.TestUtils; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import static org.yb.AssertionWrappers.*; + +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestArrayTypes extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestArrayTypes.class); + + private void assertArrayRecord(ExpectedRecordYSQL expectedRecord, + CDCSubscriber testSubscriber) throws Exception { + // For now there is just one insert in the script. + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + boolean insertRecordVerified = false; + int insertRecordCount = 0; + + for (CdcService.CDCSDKProtoRecordPB recordPB : outputList) { + if (recordPB.getRowMessage().getOp() == Op.INSERT) { + insertRecordVerified = true; + ++insertRecordCount; + ExpectedRecordYSQL.checkRecord(recordPB, expectedRecord); + } + } + + assertEquals(1, insertRecordCount); + assertTrue(insertRecordVerified); + } + + private void createTablesWithArrayTypes() throws Exception { + statement.execute("create table testvb (a int primary key, vb varbit(10)[]);"); + statement.execute("create table testboolval (a int primary key, boolval boolean[]);"); + statement.execute("create table testchval (a int primary key, chval char(5)[]);"); + statement.execute("create table testvchar (a int primary key, vchar varchar(20)[]);"); + statement.execute("create table testdt (a int primary key, dt date[]);"); + statement.execute("create table testdp (a int primary key, dp double precision[]);"); + statement.execute("create table testinetval (a int primary key, inetval inet[]);"); + statement.execute("create table testintval (a int primary key, intval integer[]);"); + statement.execute("create table testjsonval (a int primary key, jsonval json[]);"); + statement.execute("create table testjsonbval (a int primary key, jsonbval jsonb[]);"); + statement.execute("create table testmac (a int primary key, mac macaddr[]);"); + statement.execute("create table testmac8 (a int primary key, mac8 macaddr8[]);"); + statement.execute("create table testmoneyval (a int primary key, moneyval money[]);"); + statement.execute("create table testrl (a int primary key, rl real[]);"); + statement.execute("create table testsi (a int primary key, si smallint[]);"); + statement.execute("create table testtextval (a int primary key, textval text[]);"); + statement.execute("create table testtval (a int primary key, tval time[]);"); + statement.execute("create table testttzval (a int primary key, ttzval timetz[]);"); + statement.execute("create table testtimestampval (a int primary key, " + + "timestampval timestamp[]);"); + statement.execute("create table testtimestamptzval (a int primary key, " + + "timestamptzval timestamptz[]);"); + statement.execute("create table testu (a int primary key, u uuid[]);"); + statement.execute("create table testi4r (a int primary key, i4r int4range[]);"); + statement.execute("create table testi8r (a int primary key, i8r int8range[]);"); + statement.execute("create table testdr (a int primary key, dr daterange[]);"); + statement.execute("create table testtsr (a int primary key, tsr tsrange[]);"); + statement.execute("create table testtstzr (a int primary key, tstzr tstzrange[]);"); + statement.execute("create table testnr (a int primary key, nr numrange[]);"); + statement.execute("create table testbx (a int primary key, bx box[]);"); + statement.execute("create table testln (a int primary key, ln line[]);"); + statement.execute("create table testls (a int primary key, ls lseg[]);"); + statement.execute("create table testpt (a int primary key, pt point[]);"); + statement.execute("create table testcrcl (a int primary key, crcl circle[]);"); + statement.execute("create table testpoly (a int primary key, poly polygon[]);"); + statement.execute("create table testpth (a int primary key, pth path[]);"); + statement.execute("create table testinterv (a int primary key, interv interval[]);"); + statement.execute("create table testcidrval (a int primary key, cidrval cidr[]);"); + statement.execute("create table testtxid (a int primary key, txid txid_snapshot[]);"); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + } + + @Test + public void testMultiDimArrays() { + try { + assertFalse(statement.execute("drop table if exists testmulti;")); + assertFalse(statement.execute(HelperValues.createTableWithMultiDimensionalArrayColumns)); + CDCSubscriber testSubscriber = new CDCSubscriber("testmulti", getMasterAddresses()); + testSubscriber.createStream("proto"); + + String varBit = "'{{1011, 011101, 1101110111}, {1011, 011101, 1101110111}}'::varbit(10)[]"; + String booleanVal = "'{{FALSE, TRUE, TRUE, FALSE}, {FALSE, TRUE, TRUE, FALSE}}'::boolean[]"; + String charVal = "'{{\"five5\", \"five5\"}, {\"five5\", \"five5\"}}'::char(5)[]"; + String varChar = "'{{\"sample varchar\", \"test string\"}, " + + "{\"sample varchar\", \"test string\"}}'::varchar(20)[]"; + String dt = "'{{\"2021-10-07\", \"1970-01-01\"}, {\"2021-10-07\", \"1970-01-01\"}}'::date[]"; + String doublePrecision = "'{{1.23, 2.34, 3.45}, {1.23, 2.34, 3.45}}'::double precision[]"; + String inetVal = "'{{127.0.0.1, 192.168.1.1}, {127.0.0.1, 192.168.1.1}}'::inet[]"; + String integer = "'{{1, 2, 3}, {1, 2, 3}}'::integer[]"; + String jsonVal = "array[['{\"a\":\"b\"}', '{\"c\":\"d\"}'], " + + "['{\"a\":\"b\"}', '{\"c\":\"d\"}']]::json[]"; + String jsonBVal = "array[['{\"a\":\"b\"}', '{\"c\":\"d\"}'], " + + "['{\"a\":\"b\"}', '{\"c\":\"d\"}']]::jsonb[]"; + String macaddr = "'{{2c:54:91:88:c9:e3, 2c:b8:01:76:c9:e3, 2c:54:f1:88:c9:e3}, " + + "{2c:54:91:88:c9:e3, 2c:b8:01:76:c9:e3, 2c:54:f1:88:c9:e3}}'::macaddr[]"; + String macaddr8 = "'{{22:00:5c:03:55:08:01:02, 22:10:5c:03:55:d8:f1:02}, " + + "{22:00:5c:03:55:08:01:02, 22:10:5c:03:55:d8:f1:02}}'::macaddr8[]"; + String money = "'{{100.55, 200.50, 50.05}, {100.55, 200.50, 50.05}}'::money[]"; + String realVal = "'{{1.23, 4.56, 7.8901}, {1.23, 4.56, 7.8901}}'::real[]"; + String smallInt = "'{{1, 2, 3, 4, 5, 6}, {1, 2, 3, 4, 5, 6}}'::smallint[]"; + String text = "'{{\"sample1\", \"sample2\"}, {\"sample1\", \"sample2\"}}'::text[]"; + String time = "'{{12:00:32, 22:10:20, 23:59:59, 00:00:00}, " + + "{12:00:32, 22:10:20, 23:59:59, 00:00:00}}'::time[]"; + String timetz = "'{{11:00:00+05:30, 23:00:59+00, 09:59:00 UTC}, " + + "{11:00:00+05:30, 23:00:59+00, 09:59:00 UTC}}'::timetz[]"; + String timestamp = "'{{1970-01-01 0:00:10, 2000-01-01 0:00:10}, " + + "{1970-01-01 0:00:10, 2000-01-01 0:00:10}}'::timestamp[]"; + String timestamptz = "'{{1970-01-01 0:00:10+05:30, 2000-01-01 0:00:10 UTC}, " + + "{1970-01-01 0:00:10+05:30, 2000-01-01 0:00:10 UTC}}'::timestamptz[]"; + String uuid = "'{{123e4567-e89b-12d3-a456-426655440000, " + + "123e4567-e89b-12d3-a456-426655440000}, " + + "{123e4567-e89b-12d3-a456-426655440000, " + + "123e4567-e89b-12d3-a456-426655440000}}'::uuid[]"; + String i4r = "array[['(1, 5)', '(10, 100)'], ['(1, 5)', '(10, 100)']]::int4range[]"; + String i8r = "array[['(1, 10)', '(900, 10000)'], ['(1, 10)', '(900, 10000)']]::int8range[]"; + String dr = "array[['(2000-09-20, 2021-10-08)', '(1970-01-01, 2000-01-01)'], " + + "['(2000-09-20, 2021-10-08)', '(1970-01-01, 2000-01-01)']]::daterange[]"; + String tsr = "array[['(1970-01-01 00:00:00, 2000-01-01 12:00:00)', " + + "'(1970-01-01 00:00:00, 2000-01-01 12:00:00)'], " + + "['(1970-01-01 00:00:00, 2000-01-01 12:00:00)', " + + "'(1970-01-01 00:00:00, 2000-01-01 12:00:00)']]::tsrange[]"; + String tstzr = "array[['(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)', " + + "'(1970-09-14 12:30:30 UTC, 2021-10-13 09:32:30+05:30)'], " + + "['(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)', " + + "'(1970-09-14 12:30:30 UTC, 2021-10-13 09:32:30+05:30)']]::tstzrange[]"; + String nr = "array[['(10.42, 11.354)', '(-0.99, 100.9)'], " + + "['(10.42, 11.354)', '(-0.99, 100.9)']]::numrange[]"; + String box = "array[['(8,9), (1,3)', '(-1,-1), (9,27)'], " + + "['(8,9), (1,3)', '(-1,-1), (9,27)']]::box[]"; + String line = "array[['[(0, 0), (2, 5)]', '{1, 2, -10}'], " + + "['[(0, 0), (2, 5)]', '{1, 2, -10}']]::line[]"; + String lseg = "array[['[(0, 0), (2, 5)]', '[(0, 5), (6, 2)]'], " + + "['[(0, 0), (2, 5)]', '[(0, 5), (6, 2)]']]::lseg[]"; + String point = "array[['(1, 2)', '(10, 11.5)', '(0, -1)'], " + + "['(1, 2)', '(10, 11.5)', '(0, -1)']]::point[]"; + String circle = "array[['1, 2, 4', '-1, 0, 5'], ['1, 2, 4', '-1, 0, 5']]::circle[]"; + String polygon = "array[['(1, 3), (4, 12), (2, 4)', '(1, -1), (4, -12), (-2, -4)'], " + + "['(1, 3), (4, 12), (2, 4)', '(1, -1), (4, -12), (-2, -4)']]::polygon[]"; + String path = "array[['(1, 2), (10, 15), (0, 0)', '(1, 2), (10, 15), (10, 0), (-3, -2)'], " + + "['(1, 2), (10, 15), (0, 0)', '(1, 2), (10, 15), (10, 0), (-3, -2)']]::path[]"; + String interval = "array[['2020-03-10 13:47:19.7':: timestamp - " + + "'2020-03-10 12:31:13.5':: timestamp, " + + "'2020-03-10 00:00:00':: timestamp - '2020-02-10 00:00:00':: timestamp], " + + "['2020-03-10 13:47:19.7':: timestamp - '2020-03-10 12:31:13.5':: timestamp, " + + "'2020-03-10 00:00:00':: timestamp - '2020-02-10 00:00:00':: timestamp]]::interval[]"; + String cidr = "array[['12.2.0.0/22', '10.1.0.0/16'], ['12.2.0.0/22', '10.1.0.0/16']]::cidr[]"; + String txidSnapshot = "array[[txid_current_snapshot(), txid_current_snapshot()], " + + "[txid_current_snapshot(), txid_current_snapshot()]]::txid_snapshot[]"; + + String insertIntoTable = String.format(HelperValues.insertionTemplateForArrays, "testmulti", + varBit, booleanVal, charVal, varChar, dt, doublePrecision, inetVal, integer, jsonVal, + jsonBVal, macaddr, macaddr8, money, realVal, smallInt, text, time, timetz, timestamp, + timestamptz, uuid, i4r, i8r, dr, tsr, tstzr, nr, box, line, lseg, point, circle, polygon, + path, interval, cidr, txidSnapshot); + + String[] expectedRecords = HelperValues.expectedMultiDimensionalArrayColumnRecords; + + int insert = statement.executeUpdate(insertIntoTable); + assertEquals(1, insert); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + assertTrue(outputList.size() > 1); + + boolean insertRecordAsserted = false; + for (CdcService.CDCSDKProtoRecordPB record : outputList) { + if (record.getRowMessage().getOp() == CdcService.RowMessage.Op.INSERT) { + insertRecordAsserted = true; + + int tupCount = record.getRowMessage().getNewTupleCount(); + + // The first one in our case is a int value. + assertEquals(1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + + for (int i = 1; i < tupCount; ++i) { + assertEquals(expectedRecords[i-1], ExpectedRecordYSQL.getValue(record, i)); + } + } + } + assertTrue(insertRecordAsserted); + } catch (Exception e) { + LOG.error("Test to verify CDC streaming for multi-dimensional arrays failed", e); + fail(); + } + } + + @Test + public void testAllArrayTypesInSingleTable() { + try { + assertFalse(statement.execute("drop table if exists testsingle;")); + + assertFalse(statement.execute(HelperValues.createTableWithSingleDimensionalArrayColumns)); + + CDCSubscriber testSubscriber = new CDCSubscriber("testsingle", getMasterAddresses()); + testSubscriber.createStream("proto"); + + String insertIntoTable = "insert into testsingle values (1, " + + "'{1011, 011101, 1101110111}', " + + "'{FALSE, TRUE, TRUE, FALSE}', " + + "'{\"five5\", \"five5\"}', " + + "'{\"sample varchar\", \"test string\"}', " + + "'{\"2021-10-07\", \"1970-01-01\"}', " + + "'{1.23, 2.34, 3.45}', " + + "'{127.0.0.1, 192.168.1.1}', " + + "'{1, 2, 3}', " + + "array['{\"a\":\"b\"}', '{\"c\":\"d\"}']::json[], " + + "array['{\"a\":\"b\"}', '{\"c\":\"d\"}']::jsonb[], " + + "'{2c:54:91:88:c9:e3, 2c:b8:01:76:c9:e3, 2c:54:f1:88:c9:e3}', " + + "'{22:00:5c:03:55:08:01:02, 22:10:5c:03:55:d8:f1:02}', " + + "'{100.55, 200.50, 50.05}', " + + "'{1.23, 4.56, 7.8901}', " + + "'{1, 2, 3, 4, 5, 6}', " + + "'{\"sample1\", \"sample2\"}', " + + "'{12:00:32, 22:10:20, 23:59:59, 00:00:00}', " + + "'{11:00:00+05:30, 23:00:59+00, 09:59:00 UTC}', " + + "'{1970-01-01 0:00:10, 2000-01-01 0:00:10}', " + + "'{1970-01-01 0:00:10+05:30, 2000-01-01 0:00:10 UTC}', " + + "'{123e4567-e89b-12d3-a456-426655440000, 123e4567-e89b-12d3-a456-426655440000}', " + + "array['(1, 5)', '(10, 100)']::int4range[], " + + "array['(1, 10)', '(900, 10000)']::int8range[], " + + "array['(2000-09-20, 2021-10-08)', '(1970-01-01, 2000-01-01)']::daterange[], " + + "array['(1970-01-01 00:00:00, 2000-01-01 12:00:00)', '(1970-01-01 00:00:00, " + + "2000-01-01 12:00:00)']::tsrange[], " + + "array['(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)', " + + "'(1970-09-14 12:30:30 UTC, 2021-10-13 09:32:30+05:30)']::tstzrange[], " + + "array['(10.42, 11.354)', '(-0.99, 100.9)']::numrange[], " + + "array['(8,9), (1,3)', '(-1,-1), (9,27)']::box[], " + + "array['[(0, 0), (2, 5)]', '{1, 2, -10}']::line[], " + + "array['[(0, 0), (2, 5)]', '[(0, 5), (6, 2)]']::lseg[], " + + "array['(1, 2)', '(10, 11.5)', '(0, -1)']::point[], " + + "array['1, 2, 4', '-1, 0, 5']::circle[], " + + "array['(1, 3), (4, 12), (2, 4)', '(1, -1), (4, -12), (-2, -4)']::polygon[], " + + "array['(1, 2), (10, 15), (0, 0)', '(1, 2), (10, 15), (10, 0), (-3, -2)']::path[], " + + "array['2020-03-10 13:47:19.7':: timestamp - '2020-03-10 12:31:13.5':: timestamp, " + + "'2020-03-10 00:00:00':: timestamp - '2020-02-10 00:00:00':: timestamp]::interval[], " + + "array['12.2.0.0/22', '10.1.0.0/16']::cidr[], " + + "array[txid_current_snapshot(), txid_current_snapshot()]::txid_snapshot[]);"; + + int rows = statement.executeUpdate(insertIntoTable); + assertEquals(1, rows); + + String[] expectedRecords = HelperValues.expectedSingleDimensionalArrayColumnRecords; + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + assertTrue(outputList.size() > 1); + + boolean insertRecordAsserted = false; + for (CdcService.CDCSDKProtoRecordPB record : outputList) { + if (record.getRowMessage().getOp() == CdcService.RowMessage.Op.INSERT) { + insertRecordAsserted = true; + int tupCount = record.getRowMessage().getNewTupleCount(); + + // The first one in our case is a int value. + assertEquals(1, record.getRowMessage().getNewTuple(0).getDatumInt32()); + + for (int i = 1; i < tupCount; ++i) { + assertEquals(expectedRecords[i-1], ExpectedRecordYSQL.getValue(record, i)); + } + } + } + + assertTrue(insertRecordAsserted); + } catch (Exception e) { + LOG.error("Test to verify array types failed with exception", e); + fail(); + } + } + + @Test + public void testArrayTypes() { + try { + statement.execute(HelperValues.dropAllArrayTables); + createTablesWithArrayTypes(); + + CDCSubscriber vbSub = new CDCSubscriber("testvb", getMasterAddresses()); + vbSub.createStream("proto"); + CDCSubscriber boolvalSub = new CDCSubscriber("testboolval", getMasterAddresses()); + boolvalSub.createStream("proto"); + CDCSubscriber chvalSub = new CDCSubscriber("testchval", getMasterAddresses()); + chvalSub.createStream("proto"); + CDCSubscriber vcharSub = new CDCSubscriber("testvchar", getMasterAddresses()); + vcharSub.createStream("proto"); + CDCSubscriber dtSub = new CDCSubscriber("testdt", getMasterAddresses()); + dtSub.createStream("proto"); + CDCSubscriber dpSub = new CDCSubscriber("testdp", getMasterAddresses()); + dpSub.createStream("proto"); + CDCSubscriber inetvalSub = new CDCSubscriber("testinetval", getMasterAddresses()); + inetvalSub.createStream("proto"); + CDCSubscriber intvalSub = new CDCSubscriber("testintval", getMasterAddresses()); + intvalSub.createStream("proto"); + CDCSubscriber jsonvalSub = new CDCSubscriber("testjsonval", getMasterAddresses()); + jsonvalSub.createStream("proto"); + CDCSubscriber jsonbvalSub = new CDCSubscriber("testjsonbval", getMasterAddresses()); + jsonbvalSub.createStream("proto"); + CDCSubscriber macSub = new CDCSubscriber("testmac", getMasterAddresses()); + macSub.createStream("proto"); + CDCSubscriber mac8Sub = new CDCSubscriber("testmac8", getMasterAddresses()); + mac8Sub.createStream("proto"); + CDCSubscriber moneyvalSub = new CDCSubscriber("testmoneyval", getMasterAddresses()); + moneyvalSub.createStream("proto"); + CDCSubscriber rlSub = new CDCSubscriber("testrl", getMasterAddresses()); + rlSub.createStream("proto"); + CDCSubscriber siSub = new CDCSubscriber("testsi", getMasterAddresses()); + siSub.createStream("proto"); + CDCSubscriber textvalSub = new CDCSubscriber("testtextval", getMasterAddresses()); + textvalSub.createStream("proto"); + CDCSubscriber tvalSub = new CDCSubscriber("testtval", getMasterAddresses()); + tvalSub.createStream("proto"); + CDCSubscriber ttzvalSub = new CDCSubscriber("testttzval", getMasterAddresses()); + ttzvalSub.createStream("proto"); + CDCSubscriber timestampvalSub = new CDCSubscriber("testtimestampval", getMasterAddresses()); + timestampvalSub.createStream("proto"); + CDCSubscriber timestamptzvalSub = new CDCSubscriber("testtimestamptzval", + getMasterAddresses()); + timestamptzvalSub.createStream("proto"); + CDCSubscriber uSub = new CDCSubscriber("testu", getMasterAddresses()); + uSub.createStream("proto"); + CDCSubscriber i4rSub = new CDCSubscriber("testi4r", getMasterAddresses()); + i4rSub.createStream("proto"); + CDCSubscriber i8rSub = new CDCSubscriber("testi8r", getMasterAddresses()); + i8rSub.createStream("proto"); + CDCSubscriber drSub = new CDCSubscriber("testdr", getMasterAddresses()); + drSub.createStream("proto"); + CDCSubscriber tsrSub = new CDCSubscriber("testtsr", getMasterAddresses()); + tsrSub.createStream("proto"); + CDCSubscriber tstzrSub = new CDCSubscriber("testtstzr", getMasterAddresses()); + tstzrSub.createStream("proto"); + CDCSubscriber nrSub = new CDCSubscriber("testnr", getMasterAddresses()); + nrSub.createStream("proto"); + CDCSubscriber bxSub = new CDCSubscriber("testbx", getMasterAddresses()); + bxSub.createStream("proto"); + CDCSubscriber lnSub = new CDCSubscriber("testln", getMasterAddresses()); + lnSub.createStream("proto"); + CDCSubscriber lsSub = new CDCSubscriber("testls", getMasterAddresses()); + lsSub.createStream("proto"); + CDCSubscriber ptSub = new CDCSubscriber("testpt", getMasterAddresses()); + ptSub.createStream("proto"); + CDCSubscriber crclSub = new CDCSubscriber("testcrcl", getMasterAddresses()); + crclSub.createStream("proto"); + CDCSubscriber polySub = new CDCSubscriber("testpoly", getMasterAddresses()); + polySub.createStream("proto"); + CDCSubscriber pthSub = new CDCSubscriber("testpth", getMasterAddresses()); + pthSub.createStream("proto"); + CDCSubscriber intervSub = new CDCSubscriber("testinterv", getMasterAddresses()); + intervSub.createStream("proto"); + CDCSubscriber cidrvalSub = new CDCSubscriber("testcidrval", getMasterAddresses()); + cidrvalSub.createStream("proto"); + CDCSubscriber txidSub = new CDCSubscriber("testtxid", getMasterAddresses()); + txidSub.createStream("proto"); + + TestUtils.runSqlScript(connection, "sql_datatype_script/complete_array_types.sql"); + + ExpectedRecordYSQL expectedRecordVarbit = + new ExpectedRecordYSQL<>(1, "{1011,011101,1101110111}", Op.INSERT); + assertArrayRecord(expectedRecordVarbit, vbSub); + + ExpectedRecordYSQL expectedRecordBoolean = + new ExpectedRecordYSQL<>(1, "{f,t,t,f}", Op.INSERT); + assertArrayRecord(expectedRecordBoolean, boolvalSub); + + ExpectedRecordYSQL expectedRecordChar = + new ExpectedRecordYSQL<>(1, "{five5,five5}", Op.INSERT); + assertArrayRecord(expectedRecordChar, chvalSub); + + ExpectedRecordYSQL expectedRecordVarchar = + new ExpectedRecordYSQL<>(1, "{\"sample varchar\",\"test string\"}", + Op.INSERT); + assertArrayRecord(expectedRecordVarchar, vcharSub); + + ExpectedRecordYSQL expectedRecordDate = + new ExpectedRecordYSQL<>(1, "{2021-10-07,1970-01-01}", Op.INSERT); + assertArrayRecord(expectedRecordDate, dtSub); + + ExpectedRecordYSQL expectedRecordDouble = + new ExpectedRecordYSQL<>(1, "{1.23,2.34,3.45}", Op.INSERT); + assertArrayRecord(expectedRecordDouble, dpSub); + + ExpectedRecordYSQL expectedRecordInet = + new ExpectedRecordYSQL<>(1, "{127.0.0.1,192.168.1.1}", Op.INSERT); + assertArrayRecord(expectedRecordInet, inetvalSub); + + ExpectedRecordYSQL expectedRecordInteger = + new ExpectedRecordYSQL<>(1, "{1,2,3}", Op.INSERT); + assertArrayRecord(expectedRecordInteger, intvalSub); + + ExpectedRecordYSQL expectedRecordJson = + new ExpectedRecordYSQL<>(1, "{\"{\\\"a\\\":\\\"b\\\"}\"," + + "\"{\\\"c\\\":\\\"d\\\"}\"}", Op.INSERT); + assertArrayRecord(expectedRecordJson, jsonvalSub); + + ExpectedRecordYSQL expectedRecordJsonb = + new ExpectedRecordYSQL<>(1, "{\"{\\\"a\\\": \\\"b\\\"}\"," + + "\"{\\\"c\\\": \\\"d\\\"}\"}", Op.INSERT); + assertArrayRecord(expectedRecordJsonb, jsonbvalSub); + + ExpectedRecordYSQL expectedRecordMacaddr = + new ExpectedRecordYSQL<>(1, "{2c:54:91:88:c9:e3,2c:b8:01:76:c9:e3," + + "2c:54:f1:88:c9:e3}", Op.INSERT); + assertArrayRecord(expectedRecordMacaddr, macSub); + + ExpectedRecordYSQL expectedRecordMacaddr8 = + new ExpectedRecordYSQL<>(1, "{22:00:5c:03:55:08:01:02," + + "22:10:5c:03:55:d8:f1:02}", Op.INSERT); + assertArrayRecord(expectedRecordMacaddr8, mac8Sub); + + ExpectedRecordYSQL expectedRecordMoney = + new ExpectedRecordYSQL<>(1, "{$100.55,$200.50,$50.05}", Op.INSERT); + assertArrayRecord(expectedRecordMoney, moneyvalSub); + + ExpectedRecordYSQL expectedRecordReal = + new ExpectedRecordYSQL<>(1, "{1.23,4.56,7.8901}", Op.INSERT); + assertArrayRecord(expectedRecordReal, rlSub); + + ExpectedRecordYSQL expectedRecordSmallInt = + new ExpectedRecordYSQL<>(1, "{1,2,3,4,5,6}", Op.INSERT); + assertArrayRecord(expectedRecordSmallInt, siSub); + + ExpectedRecordYSQL expectedRecordText = + new ExpectedRecordYSQL<>(1, "{sample1,sample2}", Op.INSERT); + assertArrayRecord(expectedRecordText, textvalSub); + + ExpectedRecordYSQL expectedRecordTime = + new ExpectedRecordYSQL<>(1, "{12:00:32,22:10:20,23:59:59,00:00:00}", + Op.INSERT); + assertArrayRecord(expectedRecordTime, tvalSub); + + ExpectedRecordYSQL expectedRecordTimetz = + new ExpectedRecordYSQL<>(1, "{11:00:00+05:30,23:00:59+00,09:59:00+00}", + Op.INSERT); + assertArrayRecord(expectedRecordTimetz, ttzvalSub); + + ExpectedRecordYSQL expectedRecordTimestamp = + new ExpectedRecordYSQL<>(1, "{\"1970-01-01 00:00:10\",\"2000-01-01 00:00:10\"}", + Op.INSERT); + assertArrayRecord(expectedRecordTimestamp, timestampvalSub); + + ExpectedRecordYSQL expectedRecordTimestamptz = + new ExpectedRecordYSQL<>(1, "{\"1969-12-31 18:30:10+00\"," + + "\"2000-01-01 00:00:10+00\"}", Op.INSERT); + assertArrayRecord(expectedRecordTimestamptz, timestamptzvalSub); + + ExpectedRecordYSQL expectedRecordUuid = + new ExpectedRecordYSQL<>(1, "{123e4567-e89b-12d3-a456-426655440000," + + "123e4567-e89b-12d3-a456-426655440000}", Op.INSERT); + assertArrayRecord(expectedRecordUuid, uSub); + + ExpectedRecordYSQL expectedRecordI4R = + new ExpectedRecordYSQL<>(1, "{\"[2,5)\",\"[11,100)\"}", Op.INSERT); + assertArrayRecord(expectedRecordI4R, i4rSub); + + ExpectedRecordYSQL expectedRecordI8R = + new ExpectedRecordYSQL<>(1, "{\"[2,10)\",\"[901,10000)\"}", Op.INSERT); + assertArrayRecord(expectedRecordI8R, i8rSub); + + ExpectedRecordYSQL expectedRecordDateRange = + new ExpectedRecordYSQL<>(1, "{\"[2000-09-21,2021-10-08)\"," + + "\"[1970-01-02,2000-01-01)\"}", Op.INSERT); + assertArrayRecord(expectedRecordDateRange, drSub); + + ExpectedRecordYSQL expectedRecordTsRange = + new ExpectedRecordYSQL<>(1, "{\"(\\\"1970-01-01 00:00:00\\\"," + + "\\\"2000-01-01 12:00:00\\\")\",\"(\\\"1970-01-01 00:00:00\\\"," + + "\\\"2000-01-01 12:00:00\\\")\"}", Op.INSERT); + assertArrayRecord(expectedRecordTsRange, tsrSub); + + ExpectedRecordYSQL expectedRecordTstzRange = + new ExpectedRecordYSQL<>(1, "{\"(\\\"2017-07-04 12:30:30+00\\\"," + + "\\\"2021-07-04 07:00:30+00\\\")\",\"(\\\"1970-09-14 12:30:30+00\\\"," + + "\\\"2021-10-13 04:02:30+00\\\")\"}", Op.INSERT); + assertArrayRecord(expectedRecordTstzRange, tstzrSub); + + ExpectedRecordYSQL expectedRecordNumericRange = + new ExpectedRecordYSQL<>(1, "{\"(10.42,11.354)\",\"(-0.99,100.9)\"}", + Op.INSERT); + assertArrayRecord(expectedRecordNumericRange, nrSub); + + ExpectedRecordYSQL expectedRecordBox = + new ExpectedRecordYSQL<>(1, "{(8,9),(1,3);(9,27),(-1,-1)}", + Op.INSERT); + assertArrayRecord(expectedRecordBox, bxSub); + + ExpectedRecordYSQL expectedRecordLine = + new ExpectedRecordYSQL<>(1, "{\"{2.5,-1,0}\",\"{1,2,-10}\"}", + Op.INSERT); + assertArrayRecord(expectedRecordLine, lnSub); + + ExpectedRecordYSQL expectedRecordLseg = + new ExpectedRecordYSQL<>(1, "{\"[(0,0),(2,5)]\",\"[(0,5),(6,2)]\"}", + Op.INSERT); + assertArrayRecord(expectedRecordLseg, lsSub); + + ExpectedRecordYSQL expectedRecordPoint = + new ExpectedRecordYSQL<>(1, "{\"(1,2)\",\"(10,11.5)\",\"(0,-1)\"}", + Op.INSERT); + assertArrayRecord(expectedRecordPoint, ptSub); + + ExpectedRecordYSQL expectedRecordCircle = + new ExpectedRecordYSQL<>(1, "{\"<(1,2),4>\",\"<(-1,0),5>\"}", + Op.INSERT); + assertArrayRecord(expectedRecordCircle, crclSub); + + ExpectedRecordYSQL expectedRecordPolygon = + new ExpectedRecordYSQL<>(1, "{\"((1,3),(4,12),(2,4))\"," + + "\"((1,-1),(4,-12),(-2,-4))\"}", Op.INSERT); + assertArrayRecord(expectedRecordPolygon, polySub); + + ExpectedRecordYSQL expectedRecordPath = + new ExpectedRecordYSQL<>(1, "{\"((1,2),(10,15),(0,0))\"," + + "\"((1,2),(10,15),(10,0),(-3,-2))\"}", Op.INSERT); + assertArrayRecord(expectedRecordPath, pthSub); + + ExpectedRecordYSQL expectedRecordInterval = + new ExpectedRecordYSQL<>(1, "{01:16:06.2,\"29 days\"}", Op.INSERT); + assertArrayRecord(expectedRecordInterval, intervSub); + + ExpectedRecordYSQL expectedRecordCidr = + new ExpectedRecordYSQL<>(1, "{12.2.0.0/22,10.1.0.0/16}", Op.INSERT); + assertArrayRecord(expectedRecordCidr, cidrvalSub); + + ExpectedRecordYSQL expectedRecordTxid = + new ExpectedRecordYSQL<>(1, "{3:3:,3:3:}", Op.INSERT); + assertArrayRecord(expectedRecordTxid, txidSub); + } catch (Exception e) { + LOG.error("Test to verify array type values failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestBase.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestBase.java new file mode 100644 index 000000000000..dc92bd6f237b --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestBase.java @@ -0,0 +1,180 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import static org.yb.AssertionWrappers.*; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.log4j.Logger; + +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.cdc.common.ExpectedRecordYSQL; +import org.yb.cdc.util.TestUtils; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.sql.*; +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestBase extends CDCBaseClass { + private Logger LOG = Logger.getLogger(TestBase.class); + + private void executeScriptAssertRecords(ExpectedRecordYSQL[] expectedRecords, + String sqlScript) throws Exception { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + if (!sqlScript.isEmpty()) { + TestUtils.runSqlScript(connection, sqlScript); + } else { + LOG.info("No SQL script specified..."); + } + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + int expRecordIndex = 0; + int processedRecords = 0; + for (int i = 0; i < outputList.size(); ++i) { + // Ignoring the DDLs if any. + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + ExpectedRecordYSQL.checkRecord(outputList.get(i), + new ExpectedRecordYSQL<>(-1, "", Op.DDL)); + continue; + } + + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[expRecordIndex++]); + ++processedRecords; + } + // NOTE: processedRecords will be the same as expRecordIndex + assertEquals(expectedRecords.length, processedRecords); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int);"); + } + + // Check if the JDBC connection is working fine. + @Test + public void testJDBCConnection() throws SQLException { + assertFalse(connection.isClosed()); + } + + // Begin transaction, insert a row, commit. + // Expected records: 2 (INSERT, WRITE) + @Test + public void testInsertingSingleRow() { + try { + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[]{ + new ExpectedRecordYSQL<>(-1, "", Op.BEGIN), + new ExpectedRecordYSQL<>(1, 2, Op.INSERT), + new ExpectedRecordYSQL<>(-1, "", Op.COMMIT) + }; + + executeScriptAssertRecords(expectedRecords, "cdc_insert_one_row.sql"); + } catch (Exception e) { + LOG.error("Test to insert single row failed with exception: ", e); + fail(); + } + } + + // Insert a row outside transaction, try to update a row which doesn't exist. + // Expected records: 1 (INSERT) + @Test + public void testUpdateNonExistingRow() { + try { + assertFalse(statement.execute("BEGIN;")); + assertEquals(0, statement.executeUpdate("UPDATE test SET a = 32 WHERE b = 5;")); + assertFalse(statement.execute("COMMIT;")); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[]{ + new ExpectedRecordYSQL<>(1, 2, Op.INSERT) + }; + + executeScriptAssertRecords(expectedRecords, "cdc_insert_row_outside_txn.sql"); + } catch (Exception e) { + LOG.error("Test to update non-existing row failed with exception: ", e); + fail(); + } + } + + // Setup condition: Table "test" is currently empty. + // Delete a row which doesn't exist. + // Expected records: 0 + @Test + public void testDeleteNonExistingRow() { + try { + assertFalse(statement.execute("BEGIN;")); + assertEquals(0, statement.executeUpdate("DELETE FROM test WHERE b = 4;")); + assertFalse(statement.execute("COMMIT;")); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[]{ + }; + + executeScriptAssertRecords(expectedRecords, ""); + } catch (Exception e) { + LOG.error("Test to delete a non-existing row failed with exception: ", e); + fail(); + } + } + + @Test + public void testConflictsWhileInsertion() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + assertEquals(1, statement.executeUpdate("INSERT INTO test VALUES (1, 2);")); + assertFalse(statement.execute("BEGIN;")); + assertEquals(1, statement.executeUpdate("INSERT INTO test VALUES (1, 3) ON CONFLICT (a) " + + "DO UPDATE SET b = EXCLUDED.b;")); + assertFalse(statement.execute("COMMIT;")); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[]{ + new ExpectedRecordYSQL<>(1, 2, Op.INSERT), + new ExpectedRecordYSQL<>(-1, -1, Op.BEGIN), + new ExpectedRecordYSQL<>(1, 3, Op.UPDATE), + new ExpectedRecordYSQL<>(-1, -1, Op.COMMIT) + }; + + int expRecordIndex = 0; + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + ExpectedRecordYSQL.checkRecord(outputList.get(i), + new ExpectedRecordYSQL<>(-1, -1, Op.DDL)); + continue; + } + + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[expRecordIndex++]); + } + } catch (Exception e) { + LOG.error("Test failure for conflicts while insertion"); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestColumnRenaming.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestColumnRenaming.java new file mode 100644 index 000000000000..7fefa0b20a87 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestColumnRenaming.java @@ -0,0 +1,130 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; + +import java.util.ArrayList; +import java.util.List; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Test; + +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.util.YBTestRunnerNonTsanOnly; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestColumnRenaming extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestColumnRenaming.class); + + private static class ColumnName { + public String col1; + public String col2; + public CdcService.RowMessage.Op opType; + + public ColumnName(String columnOneName, String columnTwoName, + CdcService.RowMessage.Op opType) { + this.col1 = columnOneName; + this.col2 = columnTwoName; + this.opType = opType; + } + } + + private void verifyColNameInDDLRecord(CdcService.CDCSDKProtoRecordPB record, + ColumnName expectedRecord) { + CdcService.CDCSDKSchemaPB schema = record.getRowMessage().getSchema(); + + String actualCol1 = schema.getColumnInfo(0).getName(); + String actualCol2 = schema.getColumnInfo(1).getName(); + + assertEquals(expectedRecord.col1, actualCol1); + assertEquals(expectedRecord.col2, actualCol2); + } + + private void verifyColNameInInsertRecord(CdcService.CDCSDKProtoRecordPB record, + ColumnName expectedRecord) { + String actualCol1 = record.getRowMessage().getNewTuple(0).getColumnName(); + String actualCol2 = record.getRowMessage().getNewTuple(1).getColumnName(); + + assertEquals(expectedRecord.col1, actualCol1); + assertEquals(expectedRecord.col2, actualCol2); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + } + + @Test + public void testRenaming() { + try { + assertFalse(connection.isClosed()); // Check if the connection is active. + + // This will result in a DDL record (the initial header DDL). + assertFalse(statement.execute("create table test (a int primary key, b int);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + // Will result in insert records. + assertEquals(1, statement.executeUpdate("insert into test values (1, 2);")); + assertEquals(1, statement.executeUpdate("insert into test values (101, 202);")); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + // Will send a DDL. + assertFalse(statement.execute("alter table test rename column b to new_name_for_b;")); + + // Will result in insert records. + assertEquals(1, statement.executeUpdate("insert into test values (22, 33);")); + assertEquals(1, statement.executeUpdate("insert into test values (202, 303);")); + + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + ColumnName[] expectedRecords = new ColumnName[] { + new ColumnName("a", "b", Op.DDL), // initial header + new ColumnName("a", "b", Op.INSERT), // (1, 2) + new ColumnName("a", "b", Op.INSERT), // (101, 202) + new ColumnName("a", "new_name_for_b", Op.DDL), // alter. + // The above record will be added after the second call to get the changes is made. + new ColumnName("a", "new_name_for_b", Op.INSERT), // (22. 33) + new ColumnName("a", "new_name_for_b", Op.INSERT) // (202, 303) + }; + + + for (int i = 0; i < outputList.size(); ++i) { + switch (outputList.get(i).getRowMessage().getOp()) { + case DDL: + verifyColNameInDDLRecord(outputList.get(i), expectedRecords[i]); + break; + case INSERT: + verifyColNameInInsertRecord(outputList.get(i), expectedRecords[i]); + break; + } + } + + } catch (Exception e) { + LOG.error("Test to verify rename behaviour failed with exception", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestCompoundKey.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestCompoundKey.java new file mode 100644 index 000000000000..d402833c4c2c --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestCompoundKey.java @@ -0,0 +1,246 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; + +import static org.yb.AssertionWrappers.*;; +import org.junit.Before; +import org.junit.Test; + +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.cdc.common.ExpectedRecordCPKProto; +import org.yb.cdc.util.TestUtils; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestCompoundKey extends CDCBaseClass { + private Logger LOG = Logger.getLogger(TestCompoundKey.class); + + private void executeScriptAssertRecords(ExpectedRecordCPKProto[] expectedRecords, + String sqlScript) throws Exception { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + if (!sqlScript.isEmpty()) { + TestUtils.runSqlScript(connection, sqlScript); + } else { + LOG.info("No SQL script specified..."); + } + + setServerFlag(getTserverHostAndPort(), CDC_INTENT_SIZE_GFLAG, "25"); + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + int expRecordIndex = 0; + int processedRecords = 0; + for (int i = 0; i < outputList.size(); ++i) { + // Ignoring the DDLs. + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + continue; + } + + ExpectedRecordCPKProto.checkRecord(outputList.get(i), expectedRecords[expRecordIndex++]); + ++processedRecords; + } + // NOTE: processedRecords will be the same as expRecordIndex + assertEquals(expectedRecords.length, processedRecords); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int, b int, c int, d int, primary key(a, b));"); + } + + // Expected records: 1 (INSERT) + @Test + public void testInsert() { + try { + ExpectedRecordCPKProto[] expectedRecords = { + new ExpectedRecordCPKProto(1, 2, 3, 4, Op.INSERT) + }; + + executeScriptAssertRecords(expectedRecords, "compound_key_tests/cdc_compound_key.sql"); + } catch (Exception e) { + LOG.error("Test to insert compound key failed with exception: ", e); + fail(); + } + } + + @Test + public void testInsertInBatch() { + try { + ExpectedRecordCPKProto[] expectedRecords = { + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(1, 2, 3, 4, Op.INSERT), + new ExpectedRecordCPKProto(5, 6, 7, 8, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(1, 2, 4, 4, Op.UPDATE), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(5, 6, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(6, 6, 7, 8, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + }; + + executeScriptAssertRecords(expectedRecords, "compound_key_tests/cdc_cpk_insert_in_batch.sql"); + } catch (Exception e) { + LOG.error("Test to insert in batch failed with exception: ", e); + fail(); + } + } + + // Execute a script with multiple command combinations. + // Expected records: 65 (see script for more details) + @Test + public void testExecuteALongQuery() { + try { + ExpectedRecordCPKProto[] expectedRecords = { + new ExpectedRecordCPKProto(1, 2, 3, 4, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(1, 2, 4, 4, Op.UPDATE), + new ExpectedRecordCPKProto(1, 2, 4, 5, Op.UPDATE), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(1, 2, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(2, 2, 4, 5, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(2, 2, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(3, 3, 4, 5, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(3, 3, 5, 5, Op.UPDATE), + new ExpectedRecordCPKProto(3, 3, 5, 6, Op.UPDATE), + new ExpectedRecordCPKProto(7, 8, 9, 10, Op.INSERT), + new ExpectedRecordCPKProto(7, 8, 1, 10, Op.UPDATE), + new ExpectedRecordCPKProto(7, 8, 1, 11, Op.UPDATE), + new ExpectedRecordCPKProto(7, 8, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(8, 8, 10, 11, Op.INSERT), + new ExpectedRecordCPKProto(8, 8, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(9, 9, 10, 11, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(2, 3, 4, 5, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(5, 6, 7, 8, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(6, 7, 8, 9, Op.INSERT), + new ExpectedRecordCPKProto(6, 7, 17, 9, Op.UPDATE), + new ExpectedRecordCPKProto(6, 7, 17, 18, Op.UPDATE), + new ExpectedRecordCPKProto(6, 7, 26, 18, Op.UPDATE), + new ExpectedRecordCPKProto(6, 7, 26, 27, Op.UPDATE), + new ExpectedRecordCPKProto(6, 7, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(15, 7, 26, 27, Op.INSERT), + new ExpectedRecordCPKProto(15, 7, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(15, 16, 26, 27, Op.INSERT), + new ExpectedRecordCPKProto(15, 16, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(24, 25, 26, 27, Op.INSERT), + new ExpectedRecordCPKProto(24, 25, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(33, 34, 26, 27, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(33, 34, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(42, 43, 26, 27, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(60, 70, 80, 90, Op.INSERT), + new ExpectedRecordCPKProto(60, 70, 89, 90, Op.UPDATE), + new ExpectedRecordCPKProto(60, 70, 89, 99, Op.UPDATE), + new ExpectedRecordCPKProto(60, 70, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(69, 79, 89, 99, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(69, 79, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(69, 80, 90, 99, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(69, 80, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(70, 80, 90, 100, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(11, 12, 13, 14, Op.INSERT), + new ExpectedRecordCPKProto(11, 12, 14, 14, Op.UPDATE), + new ExpectedRecordCPKProto(11, 12, 14, 15, Op.UPDATE), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(12, 112, 113, 114, Op.INSERT), + new ExpectedRecordCPKProto(12, 112, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(17, 114, 115, 116, Op.INSERT), + new ExpectedRecordCPKProto(17, 114, 116, 116, Op.UPDATE), + new ExpectedRecordCPKProto(17, 114, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(18, 114, 116, 116, Op.INSERT), + new ExpectedRecordCPKProto(18, 114, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(18, 115, 117, 116, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(21, 23, 24, 25, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(41, 43, 44, 45, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.BEGIN), + new ExpectedRecordCPKProto(41, 43, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(41, 44, 45, 46, Op.INSERT), + new ExpectedRecordCPKProto(-1, -1, -1, -1, Op.COMMIT), + new ExpectedRecordCPKProto(41, 44, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(41, 44, 45, 46, Op.INSERT) + }; + + executeScriptAssertRecords(expectedRecords, "compound_key_tests/cdc_cpk_long_script.sql"); + } catch (Exception e) { + LOG.error("Test to execute a long script failed with exception: ", e); + fail(); + } + } + + // Added to verify the fix for GitHub Issue. + // [#10946] Primary key columns missing in case of compound PKs streaming with Proto format + @Test + public void testCompoundKeyInProto() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + ExpectedRecordCPKProto[] expectedRecords = { + new ExpectedRecordCPKProto(1, 2, 3, 4, Op.INSERT), + new ExpectedRecordCPKProto(0, 0, 0, 0, Op.BEGIN), + new ExpectedRecordCPKProto(5, 6, 7, 8, Op.INSERT), + new ExpectedRecordCPKProto(0, 0, 0, 0, Op.COMMIT), + new ExpectedRecordCPKProto(1, 2, 0, 0, Op.DELETE), + new ExpectedRecordCPKProto(0, 0, 0, 0, Op.BEGIN), + new ExpectedRecordCPKProto(5, 6, 8, 8, Op.UPDATE), + new ExpectedRecordCPKProto(0, 0, 0, 0, Op.COMMIT) + }; + + executeScriptAssertRecords(expectedRecords, "compound_key_tests/cdc_cpk_proto.sql"); + } catch (Exception e) { + LOG.error("Test to verify CDC behaviour with compound key and proto records failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestDDL.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestDDL.java new file mode 100644 index 000000000000..3f9b5cbadd09 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestDDL.java @@ -0,0 +1,236 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.common.*; +import org.yb.cdc.util.CDCSubscriber; + +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import static org.yb.AssertionWrappers.*; + +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestDDL extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestDDL.class); + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + } + + @Test + public void testDropColumn() { + try { + assertFalse(statement.execute("create table test (a int primary key, b int, c int);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + List outputList = new ArrayList<>(); + // We are expecting 2 DDL records, the first one with 3 columns which will be added with the + // following call. The second record will have 2 columns after a column is dropped. + testSubscriber.getResponseFromCDC(outputList); + + assertFalse(statement.execute("alter table test drop column c;")); + + // The second DDL record would be added as a result of this call. + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + // This one would contain a schema with 3 columns. + CdcService.CDCSDKProtoRecordPB ddl1 = outputList.get(0); + + // Here the schema would be only of 2 columns. + CdcService.CDCSDKProtoRecordPB ddl2 = outputList.get(1); + + assertEquals(Op.DDL, ddl1.getRowMessage().getOp()); + assertEquals(3, ddl1.getRowMessage().getSchema().getColumnInfoCount()); + assertEquals("a", ddl1.getRowMessage().getSchema().getColumnInfo(0).getName()); + assertEquals("b", ddl1.getRowMessage().getSchema().getColumnInfo(1).getName()); + assertEquals("c", ddl1.getRowMessage().getSchema().getColumnInfo(2).getName()); + + assertEquals(Op.DDL, ddl2.getRowMessage().getOp()); + assertEquals(2, ddl2.getRowMessage().getSchema().getColumnInfoCount()); + assertEquals("a", ddl2.getRowMessage().getSchema().getColumnInfo(0).getName()); + assertEquals("b", ddl2.getRowMessage().getSchema().getColumnInfo(1).getName()); + } catch (Exception e) { + LOG.error("Test to verify dropping column while CDC is attached failed", e); + fail(); + } + } + + @Test + public void testAddColumn() { + try { + assertFalse(statement.execute("create table test (a int primary key, b int);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + int dummyInsert = statement.executeUpdate("insert into test values (1, 2);"); + assertEquals(1, dummyInsert); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + assertFalse(statement.execute("alter table test add column c int;")); + + int dummyInsert2 = statement.executeUpdate("insert into test values (2, 3, 4);"); + assertEquals(1, dummyInsert2); + + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + // We expect 4 records in the outputList, the last two records will be added after the + // second call to get the changes is made. + // i.e. DDL , INSERT <1, 2>, DDL , INSERT <2, 3, 4> + + CdcService.CDCSDKProtoRecordPB ddl1 = outputList.get(0); + assertEquals(Op.DDL, ddl1.getRowMessage().getOp()); + assertEquals(2, ddl1.getRowMessage().getSchema().getColumnInfoCount()); + assertEquals("a", ddl1.getRowMessage().getSchema().getColumnInfo(0).getName()); + assertEquals("b", ddl1.getRowMessage().getSchema().getColumnInfo(1).getName()); + + ExpectedRecordYSQL insert1 = new ExpectedRecordYSQL<>(1, 2, Op.INSERT); + ExpectedRecordYSQL.checkRecord(outputList.get(1), insert1); + + CdcService.CDCSDKProtoRecordPB ddl2 = outputList.get(2); + assertEquals(Op.DDL, ddl2.getRowMessage().getOp()); + assertEquals(3, ddl2.getRowMessage().getSchema().getColumnInfoCount()); + assertEquals("a", ddl2.getRowMessage().getSchema().getColumnInfo(0).getName()); + assertEquals("b", ddl2.getRowMessage().getSchema().getColumnInfo(1).getName()); + assertEquals("c", ddl2.getRowMessage().getSchema().getColumnInfo(2).getName()); + + ExpectedRecord3Proto insert2 = new ExpectedRecord3Proto(2, 3, 4, Op.INSERT); + ExpectedRecord3Proto.checkRecord(outputList.get(3), insert2); + } catch (Exception e) { + LOG.error("Test to verify adding a column while CDC still attached failed", e); + fail(); + } + } + + @Test + public void testColumnWithDefaultValue() { + try { + assertFalse(statement.execute("create table test (a int primary key, b int default 404);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + assertEquals(1, statement.executeUpdate("insert into test values (1);")); + assertEquals(1, statement.executeUpdate("insert into test values (2, 3);")); + assertEquals(1, statement.executeUpdate("insert into test values (3);")); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, 404, Op.INSERT), + new ExpectedRecordYSQL<>(2, 3, Op.INSERT), + new ExpectedRecordYSQL<>(3, 404, Op.INSERT) + }; + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + // Ignore DDLs while comparing now. + int ind = 0; + for (int i = 0; i < expectedRecords.length; ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + continue; + } + + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[ind++]); + } + } catch (Exception e) { + LOG.error("Test to verify column with default value failed", e); + fail(); + } + } + + @Test + public void testChangeDefaultValue() { + try { + assertFalse(statement.execute("create table test (a int primary key, b int default 404);")); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + int dummyInsert = statement.executeUpdate("insert into test values (1);"); + assertEquals(1, dummyInsert); + + assertFalse(statement.execute("alter table test alter column b set default 505;")); + + int dummyInsert2 = statement.executeUpdate("insert into test values (2);"); + assertEquals(1, dummyInsert2); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, 404, Op.INSERT), + new ExpectedRecordYSQL<>(2, 505, Op.INSERT) + }; + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + int idx = 0; + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.INSERT) { + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[idx++]); + } + } + } catch (Exception e) { + LOG.error("Test to verify changing default value of columns failed", e); + fail(); + } + } + + @Test + public void testRenameTable() { + try { + final String oldTableName = "test"; + assertFalse(statement.execute(String.format("create table %s (a int primary key, b int);", + oldTableName))); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + // We expect 2 DDL records, the first one would have the old table name while the second + // one will have the new table name. The record for the first DDL record would be added + // in the following call to get the changes. + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + final String newTableName = "test_new"; + assertFalse(statement.execute(String.format("alter table test rename to %s;", newTableName))); + + // The second DDL record would be added in this call. + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + CdcService.CDCSDKProtoRecordPB ddl1 = outputList.get(0); + assertEquals(oldTableName, ddl1.getRowMessage().getTable()); + + CdcService.CDCSDKProtoRecordPB ddl2 = outputList.get(1); + assertEquals(newTableName, ddl2.getRowMessage().getTable()); + } catch (Exception e) { + LOG.error("Test to verify table rename while CDC is still attached failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestLargeTabletCount.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestLargeTabletCount.java new file mode 100644 index 000000000000..910f509a2e48 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestLargeTabletCount.java @@ -0,0 +1,78 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.common.ExpectedRecordYSQL; + +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import static org.yb.AssertionWrappers.*; + +import java.util.ArrayList; +import java.util.List; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestLargeTabletCount extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestLargeTabletCount.class); + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int) split into 12 tablets;"); + } + + @Test + public void testLargeNumberOfTablets() { + try { + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.setNumberOfTablets(12); + testSubscriber.createStream("proto"); + + int ins1 = statement.executeUpdate("insert into test values (1, 2);"); + int ins2 = statement.executeUpdate("insert into test values (22, 33);"); + + assertEquals(1, ins1); + assertEquals(1, ins2); + + ExpectedRecordYSQL[] expectedRecords = new ExpectedRecordYSQL[] { + new ExpectedRecordYSQL<>(1, 2, Op.INSERT), + new ExpectedRecordYSQL<>(22, 33, Op.INSERT) + }; + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + int idx = 0; + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.INSERT) { + ExpectedRecordYSQL.checkRecord(outputList.get(i), expectedRecords[idx++]); + } + } + } catch (Exception e) { + LOG.error("Test to check for a large number of tablets failed", e); + fail(); + } + } + +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestNullValues.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestNullValues.java new file mode 100644 index 000000000000..606faaf6bae8 --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestNullValues.java @@ -0,0 +1,386 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.*; +import org.junit.runner.RunWith; +import org.yb.Value; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.util.ArrayList; +import java.util.List; + +import static org.yb.AssertionWrappers.*; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestNullValues extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestNullValues.class); + + protected int DEFAULT_KEY_VALUE = 1; + + private void assertForNullValues(List outputList) { + assertNotNull(outputList); + + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == CdcService.RowMessage.Op.INSERT) { + CdcService.RowMessage rm = outputList.get(i).getRowMessage(); + assertEquals(DEFAULT_KEY_VALUE, rm.getNewTuple(0).getDatumInt32()); + + Value.DatumMessagePB dm = rm.getNewTuple(1); + // If it is null, no value type is going to be there. + assertFalse(dm.hasDatumBool()); + assertFalse(dm.hasDatumBytes()); + assertFalse(dm.hasDatumDouble()); + assertFalse(dm.hasDatumFloat()); + assertFalse(dm.hasDatumInt32()); + assertFalse(dm.hasDatumInt64()); + assertFalse(dm.hasDatumString()); + } + } + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + } + + private void checkNullBehaviourWithType(String type) { + try { + String createTable = String.format("create table test (a int primary key, b %s);", type); + assertFalse(statement.execute(createTable)); + + // Creating one stream. + CDCSubscriber testSubscriberProto = new CDCSubscriber(getMasterAddresses()); + testSubscriberProto.createStream("proto"); + + int dummyInsert = statement.executeUpdate(String.format("insert into test values (%d);", + DEFAULT_KEY_VALUE)); + assertEquals(1, dummyInsert); + + // Receiving response. + List outputListProto = new ArrayList<>(); + testSubscriberProto.getResponseFromCDC(outputListProto); + + assertForNullValues(outputListProto); + } catch (Exception e) { + LOG.error("Test to check for null values of column type " + type + " failed", e); + fail(); + } + } + + @Test + public void testArr() { + checkNullBehaviourWithType("int[]"); + } + + @Test + public void testBigInt() { + checkNullBehaviourWithType("bigint"); + } + + @Test + public void testBit() { + checkNullBehaviourWithType("bit(6)"); + } + + @Test + public void testVarbit() { + checkNullBehaviourWithType("varbit(6)"); + } + + @Test + public void testBoolean() { + checkNullBehaviourWithType("boolean"); + } + + @Test + public void testBox() { + checkNullBehaviourWithType("box"); + } + + @Test + public void testBytea() { + checkNullBehaviourWithType("bytea"); + } + + @Test + public void testChar() { + checkNullBehaviourWithType("char(10)"); + } + + @Test + public void testVarchar() { + checkNullBehaviourWithType("varchar(10)"); + } + + @Test + public void testCidr() { + checkNullBehaviourWithType("cidr"); + } + + @Test + public void testCircle() { + checkNullBehaviourWithType("circle"); + } + + @Test + public void testDate() { + checkNullBehaviourWithType("date"); + } + + @Test + public void testDoublePrecision() { + checkNullBehaviourWithType("double precision"); + } + + @Test + public void testInet() { + checkNullBehaviourWithType("inet"); + } + + @Test + public void testInteger() { + checkNullBehaviourWithType("int"); + } + + @Test + public void testInterval() { + checkNullBehaviourWithType("interval"); + } + + @Test + public void testJson() { + checkNullBehaviourWithType("json"); + } + + @Test + public void testJsonb() { + checkNullBehaviourWithType("jsonb"); + } + + @Test + public void testLine() { + checkNullBehaviourWithType("line"); + } + + @Test + public void testLseg() { + checkNullBehaviourWithType("lseg"); + } + + @Test + public void testMacaddr() { + checkNullBehaviourWithType("macaddr"); + } + + @Test + public void testMacaddr8() { + checkNullBehaviourWithType("macaddr8"); + } + + @Test + public void testMoney() { + checkNullBehaviourWithType("money"); + } + + @Test + public void testNumeric() { + checkNullBehaviourWithType("numeric"); + } + + @Test + public void testPath() { + checkNullBehaviourWithType("path"); + } + + @Test + public void testPoint() { + checkNullBehaviourWithType("point"); + } + + @Test + public void testPolygon() { + checkNullBehaviourWithType("polygon"); + } + + @Test + public void testReal() { + checkNullBehaviourWithType("real"); + } + + @Test + public void testSmallint() { + checkNullBehaviourWithType("smallint"); + } + + @Test + public void testInt4Range() { + checkNullBehaviourWithType("int4range"); + } + + @Test + public void testInt8Range() { + checkNullBehaviourWithType("int8range"); + } + + @Test + public void testNumRange() { + checkNullBehaviourWithType("numrange"); + } + + @Test + public void testTsrange() { + checkNullBehaviourWithType("tsrange"); + } + + @Test + public void testTstzrange() { + checkNullBehaviourWithType("tstzrange"); + } + + @Test + public void testDaterange() { + checkNullBehaviourWithType("daterange"); + } + + @Test + public void testText() { + checkNullBehaviourWithType("text"); + } + + @Test + public void testTime() { + checkNullBehaviourWithType("time"); + } + + @Test + public void testTimetz() { + checkNullBehaviourWithType("timetz"); + } + + @Test + public void testTimestamp() { + checkNullBehaviourWithType("timestamp"); + } + + @Test + public void testTimestamptz() { + checkNullBehaviourWithType("timestamptz"); + } + + @Test + public void testTsquery() { + checkNullBehaviourWithType("tsquery"); + } + + @Test + public void testTsvector() { + checkNullBehaviourWithType("tsvector"); + } + + @Test + public void testTxidSnapshot() { + checkNullBehaviourWithType("txid_snapshot"); + } + + @Test + public void testUuid() { + checkNullBehaviourWithType("uuid"); + } + + @Test + public void testPartialInsert() { + try { + final String createTable = "CREATE TABLE test (a int primary key, b int, c int, d int);"; + statement.execute(createTable); + + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStream("proto"); + + // Insert a row with value of one column + statement.execute("BEGIN;"); + statement.execute("INSERT INTO test (a, c) VALUES (1, 3);"); + statement.execute("UPDATE test SET b = 2, d = 4 WHERE a = 1;"); + statement.execute("COMMIT;"); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList); + + // This outputList should contain 6 records --> first DDL, BEGIN, INSERT, UPDATE, UPDATE + // and COMMIT. Assert for each one of the record. + + // Declaring an object to be used later. + CdcService.RowMessage rowMessage; + + // DDL record. + rowMessage = outputList.get(0).getRowMessage(); + assertEquals(Op.DDL, rowMessage.getOp()); + // There will be 4 columns in the schema. + assertEquals(4, rowMessage.getSchema().getColumnInfoCount()); + + // BEGIN record. + rowMessage = outputList.get(1).getRowMessage(); + assertEquals(Op.BEGIN, rowMessage.getOp()); + + // INSERT record. + rowMessage = outputList.get(2).getRowMessage(); + assertEquals(Op.INSERT, rowMessage.getOp()); + // Now since we have inserted only two columns (a, c), only they will have the value field, + // the other columns won't have that field since the value for them is null. + assertEquals("a", rowMessage.getNewTuple(0).getColumnName()); + assertTrue(rowMessage.getNewTuple(0).hasDatumInt32()); + assertEquals(1, rowMessage.getNewTuple(0).getDatumInt32()); + // Column b won't have any value. + assertEquals("b", rowMessage.getNewTuple(1).getColumnName()); + assertFalse(rowMessage.getNewTuple(1).hasDatumInt32()); + // Column c will have a value. + assertEquals("c", rowMessage.getNewTuple(2).getColumnName()); + assertTrue(rowMessage.getNewTuple(2).hasDatumInt32()); + assertEquals(3, rowMessage.getNewTuple(2).getDatumInt32()); + // Column d won't have any value. + assertEquals("d", rowMessage.getNewTuple(3).getColumnName()); + assertFalse(rowMessage.getNewTuple(3).hasDatumInt32()); + + // UPDATE record for update of column b. + rowMessage = outputList.get(3).getRowMessage(); + assertEquals(Op.UPDATE, rowMessage.getOp()); + assertEquals("a", rowMessage.getNewTuple(0).getColumnName()); + assertEquals(1, rowMessage.getNewTuple(0).getDatumInt32()); + assertEquals("b", rowMessage.getNewTuple(1).getColumnName()); + assertEquals(2, rowMessage.getNewTuple(1).getDatumInt32()); + + // UPDATE record for update of column d. + rowMessage = outputList.get(4).getRowMessage(); + assertEquals(Op.UPDATE, rowMessage.getOp()); + assertEquals("a", rowMessage.getNewTuple(0).getColumnName()); + assertEquals(1, rowMessage.getNewTuple(0).getDatumInt32()); + assertEquals("d", rowMessage.getNewTuple(1).getColumnName()); + assertEquals(4, rowMessage.getNewTuple(1).getDatumInt32()); + + // COMMIT record. + rowMessage = outputList.get(5).getRowMessage(); + assertEquals(Op.COMMIT, rowMessage.getOp()); + } catch (Exception e) { + LOG.error("Test to verify partial insert of columns failed"); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestSnapshot.java b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestSnapshot.java new file mode 100644 index 000000000000..81047bf2e68b --- /dev/null +++ b/java/yb-cdc/src/test/java/org/yb/cdc/ysql/TestSnapshot.java @@ -0,0 +1,264 @@ +// Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +package org.yb.cdc.ysql; + +import org.apache.log4j.Logger; +import org.junit.runner.RunWith; +import org.yb.cdc.CdcService; +import org.yb.cdc.CdcService.RowMessage.Op; +import org.yb.cdc.common.CDCBaseClass; +import org.yb.cdc.common.ExpectedRecord3Proto; +import org.yb.cdc.util.CDCSubscriber; +import org.yb.cdc.util.TestUtils; + +import java.util.*; + +import static org.yb.AssertionWrappers.*; +import org.junit.Before; +import org.junit.Test; +import org.yb.util.YBTestRunnerNonTsanOnly; + +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestSnapshot extends CDCBaseClass { + private final static Logger LOG = Logger.getLogger(TestSnapshot.class); + + protected final long DEFAULT_BATCH_SIZE = 250; + + private List convertSnapshotRecordRecordToList(CdcService.CDCSDKProtoRecordPB record) { + List valueList = new ArrayList<>(); + + int tupleSize = record.getRowMessage().getNewTupleCount(); + for (int i = 0; i < tupleSize; ++i) { + valueList.add(record.getRowMessage().getNewTuple(i).getDatumInt32()); + } + + return valueList; + } + + private void assertSnapshotRecords(Map, Boolean> expectedRecords, + List outputList) { + // Check whether the first record is a DDL. + CdcService.CDCSDKProtoRecordPB ddlRecord = outputList.get(0); + assertEquals(CdcService.RowMessage.Op.DDL, ddlRecord.getRowMessage().getOp()); + + // Start iterating from the second record of the outputList, since that will be the starting + // point for all the snapshot records. + for (int i = 1; i < outputList.size(); ++i) { + // Assuming that all snapshot records will have new_tuple value. + CdcService.CDCSDKProtoRecordPB record = outputList.get(i); + + assertEquals(CdcService.RowMessage.Op.READ, outputList.get(i).getRowMessage().getOp()); + assertTrue(outputList.get(i).getRowMessage().hasCommitTime()); + List intArr = convertSnapshotRecordRecordToList(record); + + expectedRecords.put(intArr, true); + } + + // If all values in map are true then the record pertaining to them has already been streamed. + for (Map.Entry, Boolean> pair : expectedRecords.entrySet()) { + assertTrue(pair.getValue()); + } + } + + @Before + public void setUp() throws Exception { + super.setUp(); + statement = connection.createStatement(); + statement.execute("drop table if exists test;"); + statement.execute("create table test (a int primary key, b int, c int);"); + + // Setting back to default value before each test. + try { + setServerFlag(getTserverHostAndPort(), CDC_BATCH_SIZE_GFLAG, + String.valueOf(DEFAULT_BATCH_SIZE)); + } catch (Exception e) { + LOG.error("Error while setting up default flag value for " + CDC_BATCH_SIZE_GFLAG, e); + System.exit(-1); + } + } + + private CDCSubscriber smallSnapshot() throws Exception { + // First execute a script to fill the table with some data. + TestUtils.runSqlScript(connection, "cdc_snapshot_init.sql"); + + // Check for records in snapshot response from CDC. + List outputList = new ArrayList<>(); + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + testSubscriber.createStreamAndGetSnapshot(outputList); + + Map, Boolean> expectedRecordsOutOfOrder = new HashMap<>(); + expectedRecordsOutOfOrder.put(Arrays.asList(2, 3, 4), false); + expectedRecordsOutOfOrder.put(Arrays.asList(3, 4, 5), false); + expectedRecordsOutOfOrder.put(Arrays.asList(4, 5, 404), false); + + assertSnapshotRecords(expectedRecordsOutOfOrder, outputList); + + return testSubscriber; + } + + private CDCSubscriber largeSnapshot() throws Exception { + TestUtils.runSqlScript(connection, "cdc_large_snapshot.sql"); + + List outputList = new ArrayList<>(); + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + + setServerFlag(getTserverHostAndPort(), CDC_BATCH_SIZE_GFLAG, "20000"); + + testSubscriber.createStreamAndGetSnapshot(outputList); + int insertedRecordsUsingScript = 5000; + + // +1 for the DDL record + assertEquals(insertedRecordsUsingScript+1, outputList.size()); + + Map, Boolean> expectedRecordsOutOfOrder = new HashMap<>(); + for (int i = 1; i <= insertedRecordsUsingScript; ++i) { + expectedRecordsOutOfOrder.put(Arrays.asList(i, 400, 404), false); + } + + assertSnapshotRecords(expectedRecordsOutOfOrder, outputList); + + return testSubscriber; + } + + @Test + public void testDefaultSnapshotBatchSize() { + try { + // Default batch size is 250. + TestUtils.runSqlScript(connection, "cdc_large_snapshot.sql"); + + List outputList = new ArrayList<>(); + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + + testSubscriber.createStreamAndGetSnapshot(outputList); + + // +1 if for checking the DDL record because it would be added in the beginning of + // the snapshot batch. + assertEquals(DEFAULT_BATCH_SIZE+1, outputList.size()); + } catch (Exception e) { + LOG.error("Test to verify default batch size failed", e); + fail(); + } + } + + @Test + public void testSimpleSnapshot() { + try { + smallSnapshot(); + } catch (Exception e) { + LOG.error("Test to verify the snapshot feature failed with exception", e); + fail(); + } + } + + @Test + public void testLargeSnapshot() { + try { + largeSnapshot(); + } catch (Exception e) { + LOG.error("Test to check for a large snapshot failed", e); + fail(); + } + } + + @Test + public void testSnapshotGFlag() { + try { + TestUtils.runSqlScript(connection, "cdc_large_snapshot.sql"); + + List outputList = new ArrayList<>(); + CDCSubscriber testSubscriber = new CDCSubscriber(getMasterAddresses()); + + setServerFlag(getTserverHostAndPort(), CDC_BATCH_SIZE_GFLAG, "2500"); + testSubscriber.createStreamAndGetSnapshot(outputList); + + // We get one extra record in outputList, that record is the initial DDL containing the + // schema of the table, the +1 is for the same DDL record only. + assertEquals(2500+1, outputList.size()); + } catch (Exception e) { + LOG.error("Test to verify working of GFlag for snapshots failed", e); + fail(); + } + } + + @Test + public void testSmallSnapshotThenStreaming() { + try { + CDCSubscriber testSubscriber = smallSnapshot(); + + // This statement will be executed once the snapshot is complete. + statement.execute("INSERT INTO test VALUES (10, 11, 12);"); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + boolean recordAfterSnapshotAsserted = false; + ExpectedRecord3Proto recordAfterSnapshot = new ExpectedRecord3Proto(10, 11, 12, Op.INSERT); + for (CdcService.CDCSDKProtoRecordPB record : outputList) { + if (record.getRowMessage().getOp() == Op.INSERT) { + // Since only one insert record is expected, we can assert for it. + ExpectedRecord3Proto.checkRecord(record, recordAfterSnapshot); + recordAfterSnapshotAsserted = true; + } + } + assertTrue(recordAfterSnapshotAsserted); + } catch (Exception e) { + LOG.error("Test to verify streaming after small snapshot failed", e); + fail(); + } + } + + @Test + public void testLargeSnapshotThenStreaming() { + try { + CDCSubscriber testSubscriber = largeSnapshot(); + + // These statements will be executed once the snapshot is complete. + statement.execute("INSERT INTO test VALUES (10000, 11, 12);"); + statement.execute("UPDATE test SET c = c + 10 where a = 10000;"); + statement.execute("BEGIN;"); + statement.execute("DELETE FROM test WHERE a = 10000;"); + statement.execute("COMMIT;"); + + List outputList = new ArrayList<>(); + testSubscriber.getResponseFromCDC(outputList, testSubscriber.getSubscriberCheckpoint()); + + int idx = 0; + int recordsAsserted = 0; + ExpectedRecord3Proto[] postSnapshotRecords = { + new ExpectedRecord3Proto(10000, 11, 12, Op.INSERT), + new ExpectedRecord3Proto(-1, -1, -1, Op.BEGIN), + new ExpectedRecord3Proto(10000, 11, 22, Op.UPDATE), + new ExpectedRecord3Proto(-1, -1, -1, Op.COMMIT), + new ExpectedRecord3Proto(-1, -1, -1, Op.BEGIN), + new ExpectedRecord3Proto(10000, 0, 0, Op.DELETE), + new ExpectedRecord3Proto(-1, -1, -1, Op.COMMIT) }; + + for (int i = 0; i < outputList.size(); ++i) { + if (outputList.get(i).getRowMessage().getOp() == Op.DDL) { + ExpectedRecord3Proto.checkRecord(outputList.get(i), + new ExpectedRecord3Proto(-1, -1, -1, Op.DDL)); + continue; + } + + ExpectedRecord3Proto.checkRecord(outputList.get(i), postSnapshotRecords[idx++]); + ++recordsAsserted; + } + + assertEquals(postSnapshotRecords.length, recordsAsserted); + } catch (Exception e) { + LOG.error("Test to verify streaming after large snapshot failed", e); + fail(); + } + } +} diff --git a/java/yb-cdc/src/test/resources/cdc_insert_in_batch.sql b/java/yb-cdc/src/test/resources/cdc_insert_in_batch.sql new file mode 100644 index 000000000000..bdc1287eb467 --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_insert_in_batch.sql @@ -0,0 +1,4 @@ +begin; +insert into test values (7, 8, 9); +insert into test values (4, 5, 6), (34, 35, 45), (1000, 1001, 1004); +end; diff --git a/java/yb-cdc/src/test/resources/cdc_insert_in_batch_outside_txn.sql b/java/yb-cdc/src/test/resources/cdc_insert_in_batch_outside_txn.sql new file mode 100644 index 000000000000..95be1d1569f8 --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_insert_in_batch_outside_txn.sql @@ -0,0 +1,2 @@ +insert into test values (7, 8, 9); +insert into test values (4, 5, 6), (34, 35, 45), (1000, 1001, 1004); diff --git a/java/yb-cdc/src/test/resources/cdc_insert_one_row.sql b/java/yb-cdc/src/test/resources/cdc_insert_one_row.sql new file mode 100644 index 000000000000..f68504586280 --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_insert_one_row.sql @@ -0,0 +1,3 @@ +begin; +insert into test values (1, 2); +commit; diff --git a/java/yb-cdc/src/test/resources/cdc_insert_row_outside_txn.sql b/java/yb-cdc/src/test/resources/cdc_insert_row_outside_txn.sql new file mode 100644 index 000000000000..d7f52e6d7f4c --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_insert_row_outside_txn.sql @@ -0,0 +1,4 @@ +--DROP TABLE IF EXISTS test; +--CREATE TABLE test (a int primary key, b int); + +INSERT INTO test VALUES (1, 2); diff --git a/java/yb-cdc/src/test/resources/cdc_large_snapshot.sql b/java/yb-cdc/src/test/resources/cdc_large_snapshot.sql new file mode 100644 index 000000000000..1b86121e357d --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_large_snapshot.sql @@ -0,0 +1 @@ +do $$ begin for i in 1..5000 loop insert into test values (i, 400, 404); end loop; end; $$; diff --git a/java/yb-cdc/src/test/resources/cdc_long_script.sql b/java/yb-cdc/src/test/resources/cdc_long_script.sql new file mode 100644 index 000000000000..0c40e64b3db4 --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_long_script.sql @@ -0,0 +1,80 @@ +insert into test values (1,2,3); + +update test set c=c+1 where a=1; + +update test set a=a+1 where a=1; + +begin; +insert into test values(7,8,9); +rollback; + +begin; +insert into test values(7,8,9); +update test set b=b+9 where a=7; +end transaction; + +begin; +insert into test values(6,7,8); +update test set c=c+9 where a=6; +update test set a=a+9 where a=6; +commit; + +update test set b=b+1, c=c+1 where a=1; + +begin; +update test set b=b+1, c=c+1 where a=1; +end; + +begin; +insert into test values(11,12,13); +update test set b=b+1, c=c+1 where a=11; +end; + +begin; +insert into test values(12,112,113); +delete from test where a=12; +end; + +begin; +insert into test values(13,113,114); +update test set c=c+1 where a=13; +update test set a=a+1 where a=13; +end; + +begin; +insert into test values(17,114,115); +update test set c=c+1 where a=17; +update test set a=a+1 where a=17; +update test set b=b+1, c=c+1 where a=18; +end; + +begin; +update test set b=b+1, c=c+1 where a=18; +insert into test values(20,21,22); +update test set b=b+1, c=c+1 where a=20; +update test set a=a+1 where a=20; +end; + +begin; +update test set b=b+1, c=c+1 where a=21; +delete from test where a=21; +insert into test values(21,23,24); +end; + +begin; +insert into test values (-1,-2,-3), (-4,-5,-6); +insert into test values (-11, -12, -13); +delete from test where a=-1; +commit; + +insert into test values (404, 405, 406), (104, 204, 304); + +insert into test values(41,43,44); + +update test set b=b+1, c=c+1 where a=41; + +delete from test where a=41; + +insert into test values(41,43,44); + +update test set b=b+1, c=c+1 where a=41; diff --git a/java/yb-cdc/src/test/resources/cdc_long_txn_rollback.sql b/java/yb-cdc/src/test/resources/cdc_long_txn_rollback.sql new file mode 100644 index 000000000000..4b38cd2817cb --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_long_txn_rollback.sql @@ -0,0 +1,10 @@ +--drop table if exists test; +--create table test (a int primary key, b int, c int); + +begin; +insert into test values (7, 8, 9); +update test set b = b + 1, c = c + 1 where a = 7; +insert into test values (4, 5, 6), (34, 35, 45), (1000, 1001, 1004); +insert into test values (32, 12, 20); +delete from test where c = 10; +rollback; diff --git a/java/yb-cdc/src/test/resources/cdc_snapshot_init.sql b/java/yb-cdc/src/test/resources/cdc_snapshot_init.sql new file mode 100644 index 000000000000..95f4bef2e837 --- /dev/null +++ b/java/yb-cdc/src/test/resources/cdc_snapshot_init.sql @@ -0,0 +1,15 @@ +insert into test values (1, 2, 3); +insert into test values (2, 3, 4); +insert into test values (3, 4, 5); + +begin; +insert into test values (4, 5, 6); +commit; + +begin; +delete from test where a = 1; +commit; + +begin; +update test set c = 404 where a = 4; +commit; diff --git a/java/yb-cdc/src/test/resources/compound_key_tests/cdc_compound_key.sql b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_compound_key.sql new file mode 100644 index 000000000000..424a83a9d0b6 --- /dev/null +++ b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_compound_key.sql @@ -0,0 +1,4 @@ +--drop table if exists test; +--create table test (a int, b int, c int, d int, primary key (a, b)); + +insert into test values (1, 2, 3, 4); diff --git a/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_insert_in_batch.sql b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_insert_in_batch.sql new file mode 100644 index 000000000000..4a585dfbf7f2 --- /dev/null +++ b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_insert_in_batch.sql @@ -0,0 +1,7 @@ +--drop table if exists test; +--create table test (a int, b int, c int, d int, primary key(a, b)); + +insert into test values (1, 2, 3, 4), (5, 6, 7, 8); + +update test set c = c + 1 where a = 1 and b = 2; +update test set a = a + 1 where c = 7; diff --git a/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_long_script.sql b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_long_script.sql new file mode 100644 index 000000000000..bb63cc1399e6 --- /dev/null +++ b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_long_script.sql @@ -0,0 +1,93 @@ +--drop table if exists test; +--create table test(a int, b int,c int, d int, primary key(a,b)); + +insert into test values(1,2,3,4); + +update test set c=c+1, d=d+1 where a=1; + +update test set a=a+1 where a=1; + +update test set a=a+1, b=b+1 where a=2; + +begin; +update test set c=c+1, d=d+1 where a=3; +insert into test values(7,8,9,10); +update test set c=c+1, d=d+1 where a=7; +update test set a=a+1 where a=7; +update test set a=a+1, b=b+1 where a=8; +end; + +begin; +insert into test values(2,3,4,5); +end; + +begin; +insert into test values(5,6,7,8); +end; + +begin; +insert into test values(6,7,8,9); +update test set d=d+9, c=c+9 where a=6; +update test set d=d+9, c=c+9 where a=6 and b=7; +update test set a=a+9 where a=6; +update test set b=b+9 where a=15 and b=7; +update test set a=a+9, b=b+9 where a=15; +update test set a=a+9, b=b+9 where a=24 and b=25; +commit; + +update test set a=a+9, b=b+9 where a=33 and b=34; + +begin; +insert into test values(60,70,80,90); +update test set c=c+9, d=d+9 where a=60; +update test set a=a+9, b=b+9 where a=60; +commit; +update test set b=b+1, c=c+1 where a=69; + +begin; +update test set a=a+1, d=d+1 where a=69; +end; + +begin; +insert into test values(11,12,13,14); +update test set d=d+1, c=c+1 where a=11; +end; + +begin; +insert into test values(12,112,113,114); +delete from test where a=12 and b=112; +end; + +begin; +insert into test values(13,113,114,115); +update test set c=c+1, d=d+1 where a=13; +update test set a=a+1, b=b+1 where a=13; +rollback; + +begin; +insert into test values(17,114,115, 116); +update test set c=c+1 where a=17; +update test set a=a+1 where a=17; +update test set b=b+1, c=c+1 where a=18; +end; + +begin; +update test set b=b+1, c=c+1 where a=18; +insert into test values(20,21,22,23); +update test set d=d+1, c=c+1 where a=20; +update test set a=a+1, b=b+1 where a=20; +abort; + +begin; +update test set b=b+1, c=c+1 where a=21; +delete from test where a=21 and b=23; +insert into test values(21,23,24,25); +end; + +insert into test values(41,43,44,45); + +update test set b=b+1, c=c+1, d=d+1 where a=41; + +delete from test where a=41 and b=44; + +insert into test values(41,44,45,46); diff --git a/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_proto.sql b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_proto.sql new file mode 100644 index 000000000000..186751e0d949 --- /dev/null +++ b/java/yb-cdc/src/test/resources/compound_key_tests/cdc_cpk_proto.sql @@ -0,0 +1,9 @@ +insert into test values (1, 2, 3, 4); + +begin; +insert into test values (5, 6, 7, 8); +commit; + +delete from test where a = 1 and b = 2; + +update test set c = c + 1 where a = 5; diff --git a/java/yb-cdc/src/test/resources/config.properties b/java/yb-cdc/src/test/resources/config.properties new file mode 100644 index 000000000000..9c6ed2ff8099 --- /dev/null +++ b/java/yb-cdc/src/test/resources/config.properties @@ -0,0 +1,12 @@ +#null +#Mon Nov 01 20:56:35 IST 2021 +admin.operation.timeout.ms=30000 +operation.timeout.ms=30000 +num.io.threads=1 +socket.read.timeout.ms=30000 +table.name=test +stream.id= +schema.name=yugabyte +format=json +master.addrs=127.0.0.1\:7100 +tablets.list= diff --git a/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_drop.sql b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_drop.sql new file mode 100644 index 000000000000..6da894d759cc --- /dev/null +++ b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_drop.sql @@ -0,0 +1,14 @@ +--drop table if exists test; +--create table test (a int primary key, b int); + +insert into test values (1, 2); +insert into test values (2, 3), (4, 5); + +begin; +update test set b = b + 1 where a = 4; -- the row becomes (4, 6) +delete from test where a = 2; +commit; + +begin; +alter table test drop column b; +commit; diff --git a/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_rename.sql b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_rename.sql new file mode 100644 index 000000000000..be1180f34a4a --- /dev/null +++ b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_rename.sql @@ -0,0 +1,10 @@ +--drop table if exists test; +--create table test (a int primary key, b int); + +insert into test values (4, 5), (6, 7), (8, 9), (1, 2), (10, 11); + +begin; +update test set b = 0 where a in (4, 6); +commit; + +alter table test rename b to c; diff --git a/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_truncate.sql b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_truncate.sql new file mode 100644 index 000000000000..c060e2a3ac9e --- /dev/null +++ b/java/yb-cdc/src/test/resources/ddl_test_scripts/cdc_ddl_truncate.sql @@ -0,0 +1,16 @@ +--drop table if exists test; +--create table test (a int primary key, b int); + +insert into test values (1, 2); + +begin; +insert into test values (5, 6); +commit; + +insert into test values (7, 8); + +begin; +insert into test values (0, 1); +end; + +truncate table test; diff --git a/java/yb-cdc/src/test/resources/sql_datatype_script/complete_array_types.sql b/java/yb-cdc/src/test/resources/sql_datatype_script/complete_array_types.sql new file mode 100644 index 000000000000..43bba73a783b --- /dev/null +++ b/java/yb-cdc/src/test/resources/sql_datatype_script/complete_array_types.sql @@ -0,0 +1,73 @@ +insert into testvb values (1, '{1011, 011101, 1101110111}'); + +insert into testboolval values (1, '{FALSE, TRUE, TRUE, FALSE}'); + +insert into testchval values (1, '{"five5", "five5"}'); + +insert into testvchar values (1, '{"sample varchar", "test string"}'); + +insert into testdt values (1, '{"2021-10-07", "1970-01-01"}'); + +insert into testdp values (1, '{1.23, 2.34, 3.45}'); + +insert into testinetval values (1, '{127.0.0.1, 192.168.1.1}'); + +insert into testintval values (1, '{1, 2, 3}'); + +insert into testjsonval values (1, array['{"a":"b"}', '{"c":"d"}']::json[]); + +insert into testjsonbval values (1, array['{"a":"b"}', '{"c":"d"}']::jsonb[]); + +insert into testmac values (1, '{2c:54:91:88:c9:e3, 2c:b8:01:76:c9:e3, 2c:54:f1:88:c9:e3}'); + +insert into testmac8 values (1, '{22:00:5c:03:55:08:01:02, 22:10:5c:03:55:d8:f1:02}'); + +insert into testmoneyval values (1, '{100.55, 200.50, 50.05}'); + +insert into testrl values (1, '{1.23, 4.56, 7.8901}'); + +insert into testsi values (1, '{1, 2, 3, 4, 5, 6}'); + +insert into testtextval values (1, '{"sample1", "sample2"}'); + +insert into testtval values (1, '{12:00:32, 22:10:20, 23:59:59, 00:00:00}'); + +insert into testttzval values (1, '{11:00:00+05:30, 23:00:59+00, 09:59:00 UTC}'); + +insert into testtimestampval values (1, '{1970-01-01 0:00:10, 2000-01-01 0:00:10}'); + +insert into testtimestamptzval values (1, '{1970-01-01 0:00:10+05:30, 2000-01-01 0:00:10 UTC}'); + +insert into testu values (1, '{123e4567-e89b-12d3-a456-426655440000, 123e4567-e89b-12d3-a456-426655440000}'); + +insert into testi4r values (1, array['(1, 5)', '(10, 100)']::int4range[]); + +insert into testi8r values (1, array['(1, 10)', '(900, 10000)']::int8range[]); + +insert into testdr values (1, array['(2000-09-20, 2021-10-08)', '(1970-01-01, 2000-01-01)']::daterange[]); + +insert into testtsr values (1, array['(1970-01-01 00:00:00, 2000-01-01 12:00:00)', '(1970-01-01 00:00:00, 2000-01-01 12:00:00)']::tsrange[]); + +insert into testtstzr values (1, array['(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)', '(1970-09-14 12:30:30 UTC, 2021-10-13 09:32:30+05:30)']::tstzrange[]); + +insert into testnr values (1, array['(10.42, 11.354)', '(-0.99, 100.9)']::numrange[]); + +insert into testbx values (1, array['(8,9), (1,3)', '(-1,-1), (9,27)']::box[]); + +insert into testln values (1, array['[(0, 0), (2, 5)]', '{1, 2, -10}']::line[]); + +insert into testls values (1, array['[(0, 0), (2, 5)]', '[(0, 5), (6, 2)]']::lseg[]); + +insert into testpt values (1, array['(1, 2)', '(10, 11.5)', '(0, -1)']::point[]); + +insert into testcrcl values (1, array['1, 2, 4', '-1, 0, 5']::circle[]); + +insert into testpoly values (1, array['(1, 3), (4, 12), (2, 4)', '(1, -1), (4, -12), (-2, -4)']::polygon[]); + +insert into testpth values (1, array['(1, 2), (10, 15), (0, 0)', '(1, 2), (10, 15), (10, 0), (-3, -2)']::path[]); + +insert into testinterv values (1, array['2020-03-10 13:47:19.7':: timestamp - '2020-03-10 12:31:13.5':: timestamp, '2020-03-10 00:00:00':: timestamp - '2020-02-10 00:00:00':: timestamp]::interval[]); + +insert into testcidrval values (1, array['12.2.0.0/22', '10.1.0.0/16']::cidr[]); + +insert into testtxid values (1, array[txid_current_snapshot(), txid_current_snapshot()]::txid_snapshot[]); diff --git a/java/yb-cdc/src/test/resources/sql_datatype_script/complete_datatype_test.sql b/java/yb-cdc/src/test/resources/sql_datatype_script/complete_datatype_test.sql new file mode 100644 index 000000000000..93bf6c52f7db --- /dev/null +++ b/java/yb-cdc/src/test/resources/sql_datatype_script/complete_datatype_test.sql @@ -0,0 +1,266 @@ +-- bit +insert into testbit values (1, '001111'); +insert into testbit values (2, '110101'); +begin; +insert into testbit values (3, '111111'); +update testbit set b = '000000', a = 0 where a = 1; +commit; +delete from testbit where a = 2; + +-- boolean +insert into testboolean values (1, FALSE); +insert into testboolean values (3, TRUE); +begin; +update testboolean set b = FALSE where a = 3; +commit; +delete from testboolean where a = 1; + +-- box +insert into testbox values (1, '(8, 9), (1, 3)'); +update testbox set b = '(8, 9), (10, 31)' where a = 1; +delete from testbox where a = 1; +begin; +insert into testbox values (2, '(8, 9), (10, 31)'); +commit; + +-- bytea +insert into testbytea values (1, E'\\001'); +update testbytea set b = E'\\xDEADBEEF' where a = 1; +delete from testbytea where a = 1; +begin; +insert into testbytea values (2, E'\\xDEADBEEF'); +commit; + +-- cidr +insert into testcidr values (1, '10.1.0.0/16'); +update testcidr set b = '12.2.0.0/22' where a = 1; +delete from testcidr where a = 1; +begin; +insert into testcidr values (2, '12.2.0.0/22'); +commit; + +-- circle +insert into testcircle values (10, '2, 3, 32'); +update testcircle set b = '0, 0, 10' where a = 10; +delete from testcircle where a = 10; +begin; +insert into testcircle values (1000, '0, 0, 4'); +commit; + +-- date +insert into testdate values (1, '2021-09-20'); +update testdate set b = '2021-09-29' where a = 1; +insert into testdate values (2, '2000-01-01'); +delete from testdate where a = 2; +insert into testdate values (3, '1970-01-01'); +update testdate set a = a + 1 where a = 3; + +-- double +insert into testdouble values (1, 10.42); +insert into testdouble values (3, 0.5); +begin; +update testdouble set b = 34.56 where a = 5; +commit; +update testdouble set a = a + 1 where a = 3; + +-- inet +insert into testinet values (1, '127.0.0.1'); +insert into testinet values (2, '0.0.0.0'); +insert into testinet values (3, '192.168.1.1'); +delete from testinet where a = 3; + +-- int +insert into testint values (1, 2); +insert into testint values (3, 4); +begin; +update testint set b = b + 1 where a = 3; +commit; +begin; +insert into testint values (7, 8); +update testint set a = a + 1 where a = 7; +end; +delete from testint where a = 8; + +-- json +insert into testjson values (1, '{"first_name":"vaibhav"}'); +insert into testjson values (2, '{"last_name":"kushwaha"}'); +update testjson set b = '{"name":"vaibhav kushwaha"}' where a = 2; +begin; +delete from testjson where a = 1; +insert into testjson values (3, '{"a":97, "b":"98"}'); +commit; + +-- jsonb +insert into testjsonb values (1, '{"first_name":"vaibhav"}'); +insert into testjsonb values (2, '{"last_name":"kushwaha"}'); +update testjsonb set b = '{"name":"vaibhav kushwaha"}' where a = 2; +begin; +delete from testjsonb where a = 1; +insert into testjsonb values (3, '{"a":97, "b":"98"}'); +commit; + +-- line +insert into testline values (1, '{1, 2, -8}'); +update testline set b = '{1, 1, -5}' where a = 1; +delete from testline where a = 1; +begin; +insert into testline values (29, '[(0, 0), (2, 5)]'); +commit; + +-- lseg +insert into testlseg values (1, '[(0, 0), (2, 4)]'); +update testlseg set b = '((-1, -1), (10, -8))' where a = 1; +delete from testlseg where a = 1; +begin; +insert into testlseg values (37, '[(1, 3), (3, 5)]'); +commit; + +-- macaddr8 +insert into testmacaddr8 values (1, '22:00:5c:03:55:08:01:02'); +update testmacaddr8 set b = '22:00:5c:04:55:08:01:02' where a = 1; +begin; +insert into testmacaddr8 values (2, '22:00:5c:03:55:08:01:02'); +delete from testmacaddr8 where a = 2; +commit; +insert into testmacaddr8 values (3, '22:00:5c:05:55:08:01:02'); +delete from testmacaddr8 where a = 3; + +-- macaddr +insert into testmacaddr values (1, '2C:54:91:88:C9:E3'); +update testmacaddr set b = '2C:54:91:E8:99:D2' where a = 1; +delete from testmacaddr where a = 1; +begin; +insert into testmacaddr values (2, '2C:54:91:E8:99:D2'); +commit; + +-- money +insert into testmoney values (1, 100.5); +insert into testmoney values (2, 10.12); +begin; +insert into testmoney values (3, 1.23); +update testmoney set b = b - '$10' where a = 1; +commit; +delete from testmoney where a = 2; + +-- numeric +insert into testnumeric values (1, 20.5); +insert into testnumeric values (2, 100.75); +begin; +insert into testnumeric values (3, 3.456); +commit; + +-- path +insert into testpath values (23, '(1, 2), (20, -10)'); +update testpath set b = '(-1, -1)' where a = 23; +delete from testpath where a = 23; +begin; +insert into testpath values (34, '(0, 0), (3, 4), (5, 5), (1, 2)'); +commit; + +-- point +insert into testpoint values (11, '(0, -1)'); +update testpoint set b = '(1, 3)' where a = 11; +delete from testpoint where a = 11; +begin; +insert into testpoint values (21, '(33, 44)'); +commit; + +-- polygon +insert into testpolygon values (1, '(1, 3), (4, 12), (2, 4)'); +update testpolygon set b = '(1, 3), (4, 12), (2, 4), (1, 4)' where a = 1; +delete from testpolygon where a = 1; +begin; +insert into testpolygon values (27, '(1, 3), (2, 4), (1, 4)'); +commit; + +-- text +insert into testtext values (1, 'sample string with pk 1'); +insert into testtext values (3, 'sample string with pk 3'); +begin; +update testtext set a = a + 1, b = 'sample string with pk 2' where a = 1; +commit; +update testtext set b = 'random sample string' where a = 3; + +-- time +insert into testtime values (1, '11:30:59'); +update testtime set b = '23:30:59' where a = 1; +begin; +insert into testtime values (2, '00:00:01'); +update testtime set b = b + '00:00:59' where a = 2; +commit; +delete from testtime where a = 1; +delete from testtime where a = 2; + +-- timestamp +insert into testtimestamp values (1, '2017-07-04 12:30:30'); +insert into testtimestamp values (2, '2021-09-29 00:00:00'); +update testtimestamp set b = '1970-01-01 00:00:10' where a = 1; + +-- timetz +insert into testtimetz values (1, '11:30:59+05:30'); +update testtimetz set b = '23:30:59+05:30' where a = 1; +begin; +insert into testtimetz values (2, '00:00:01 UTC'); +commit; +delete from testtimetz where a = 1; +delete from testtimetz where a = 2; + +-- uuid +insert into testuuid values (1, 'ffffffff-ffff-ffff-ffff-ffffffffffff'); +insert into testuuid values (3, 'ffffffff-ffff-ffff-ffff-ffffffffffff'); +begin; +update testuuid set b = '123e4567-e89b-12d3-a456-426655440000' where a = 3; +commit; +delete from testuuid where a = 1; +insert into testuuid values (2, '123e4567-e89b-12d3-a456-426655440000'); + +-- varbit +insert into testvarbit values (1, '001111'); +insert into testvarbit values (2, '1101011101'); +begin; +insert into testvarbit values (3, '11'); +update testvarbit set b = '0', a = 0 where a = 1; +commit; +delete from testvarbit where a = 2; + +-- timestamptz +insert into testtstz values (1, '1970-01-01 00:10:00+05:30'); +begin; +update testtstz set b = '2022-01-01 00:10:00+05:30' where a = 1; +delete from testtstz where a = 1; +commit; + +-- int4range +insert into testint4range values (1, '(4, 14)'); +begin; +update testint4range set b = '(5, 43)' where a = 1; +delete from testint4range where a = 1; +commit; + +-- int8range +insert into testint8range values (1, '(4, 15)'); +begin; +update testint8range set b = '(1, 100000)' where a = 1; +delete from testint8range where a = 1; +commit; + +-- tsrange +insert into testtsrange values (1, '(1970-01-01 00:00:00, 2000-01-01 12:00:00)'); +begin; +update testtsrange set b = '(1970-01-01 00:00:00, 2022-11-01 12:00:00)' where a = 1; +delete from testtsrange where a = 1; +commit; + +-- tstzrange +insert into testtstzrange values (1, '(2017-07-04 12:30:30 UTC, 2021-07-04 12:30:30+05:30)'); +begin; +update testtstzrange set b = '(2017-07-04 12:30:30 UTC, 2021-10-04 12:30:30+05:30)' where a = 1; +delete from testtstzrange where a = 1; +commit; + +-- daterange +insert into testdaterange values (1, '(2019-10-07, 2021-10-07)'); +begin; +update testdaterange set b = '(2019-10-07, 2020-10-07)' where a = 1; +delete from testdaterange where a = 1; +commit; diff --git a/java/yb-client/.gitignore b/java/yb-client/.gitignore index 4bc0ed38b939..38bf35eb0e94 100644 --- a/java/yb-client/.gitignore +++ b/java/yb-client/.gitignore @@ -1,3 +1,4 @@ # Maven build artifacts dependency-reduced-pom.xml +demoApplication.log diff --git a/java/yb-client/pom.xml b/java/yb-client/pom.xml index 94e81d259769..81b7f37a3d23 100644 --- a/java/yb-client/pom.xml +++ b/java/yb-client/pom.xml @@ -94,6 +94,11 @@ org.slf4j slf4j-api + + org.postgresql + postgresql + 42.2.14 + commons-io commons-io @@ -200,6 +205,25 @@ + + org.apache.maven.plugins + maven-assembly-plugin + 3.1.1 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + org.apache.maven.plugins maven-jar-plugin diff --git a/java/yb-client/src/main/java/org/yb/Type.java b/java/yb-client/src/main/java/org/yb/Type.java index 5dae5e4daf16..70ac6a8392ee 100644 --- a/java/yb-client/src/main/java/org/yb/Type.java +++ b/java/yb-client/src/main/java/org/yb/Type.java @@ -161,6 +161,7 @@ public static Type getTypeForDataType(DataType type) { case INT16: return INT16; case INT32: return INT32; case INT64: return INT64; + case UINT64: return INT64; case STRING: return STRING; case BOOL: return BOOL; case FLOAT: return FLOAT; diff --git a/java/yb-client/src/main/java/org/yb/client/AsyncYBClient.java b/java/yb-client/src/main/java/org/yb/client/AsyncYBClient.java index 00eaab02e68f..c422a9f820a5 100644 --- a/java/yb-client/src/main/java/org/yb/client/AsyncYBClient.java +++ b/java/yb-client/src/main/java/org/yb/client/AsyncYBClient.java @@ -60,6 +60,7 @@ import java.net.SocketAddress; import java.net.UnknownHostException; import java.nio.charset.Charset; +import java.rmi.Remote; import java.security.KeyFactory; import java.security.KeyStore; import java.security.PrivateKey; @@ -112,6 +113,7 @@ import org.yb.Schema; import org.yb.annotations.InterfaceAudience; import org.yb.annotations.InterfaceStability; +import org.yb.cdc.CdcService; import org.yb.consensus.Metadata; import org.yb.master.CatalogEntityInfo; import org.yb.master.MasterClientOuterClass; @@ -167,6 +169,11 @@ public class AsyncYBClient implements AutoCloseable { public static final long DEFAULT_OPERATION_TIMEOUT_MS = 10000; public static final long DEFAULT_SOCKET_READ_TIMEOUT_MS = 5000; + public static final int DEFAULT_MAX_TABLETS = MasterClientOuterClass + .GetTableLocationsRequestPB + .getDefaultInstance() + .getMaxReturnedLocations(); + private final ClientSocketChannelFactory channelFactory; // TODO(Bharat) - get tablet id from master leader. @@ -241,7 +248,7 @@ public class AsyncYBClient implements AutoCloseable { /** * Timestamp required for HybridTime external consistency through timestamp * propagation. - * @see src/yb/common/common.proto + * @see "src/yb/common/common.proto" */ private long lastPropagatedTimestamp = NO_TIMESTAMP; @@ -277,6 +284,8 @@ public class AsyncYBClient implements AutoCloseable { private volatile boolean closed; + private final int numTabletsInTable; + private AsyncYBClient(AsyncYBClientBuilder b) { this.channelFactory = b.createChannelFactory(); this.masterAddresses = b.masterAddresses; @@ -290,6 +299,7 @@ private AsyncYBClient(AsyncYBClientBuilder b) { this.clientHost = b.clientHost; this.clientPort = b.clientPort; this.defaultSocketReadTimeoutMs = b.defaultSocketReadTimeoutMs; + this.numTabletsInTable = b.numTablets; } /** @@ -365,13 +375,21 @@ public Deferred getMasterAddresses(final HostAndPort * @param tableId the table id to subscribe to. * @return a deferred object for the response from server. */ - public Deferred createCDCStream(final HostAndPort hp, String tableId) { + public Deferred createCDCStream(final HostAndPort hp, + String tableId, + String nameSpaceName, + String format, + String checkpointType) { checkIsClosed(); TabletClient client = newSimpleClient(hp); if (client == null) { throw new IllegalStateException("Could not create a client to " + hp.toString()); } - CreateCDCStreamRequest rpc = new CreateCDCStreamRequest(this.masterTable, tableId); + CreateCDCStreamRequest rpc = new CreateCDCStreamRequest(this.masterTable, + tableId, + nameSpaceName, + format, + checkpointType); rpc.setTimeoutMillis(defaultAdminOperationTimeoutMs); Deferred d = rpc.getDeferred(); rpc.attempt++; @@ -379,30 +397,88 @@ public Deferred createCDCStream(final HostAndPort hp, S return d; } + public Deferred createCDCStream(YBTable table, + String nameSpaceName, + String format, + String checkpointType) { + checkIsClosed(); + CreateCDCStreamRequest rpc = new CreateCDCStreamRequest(table, + table.getTableId(), + nameSpaceName, + format, + checkpointType); + rpc.setTimeoutMillis(defaultAdminOperationTimeoutMs); + Deferred d = rpc.getDeferred().addErrback(new Callback() { + @Override + public Object call(Object o) throws Exception { + return o; + } + }); + sendRpcToTablet(rpc); + return d; + } + /** * Get changes for a given tablet and stream. - * @param hp host port of the server. * @param table the table to get changes for. * @param streamId the stream to get changes for. * @param tabletId the tablet to get changes for. * @param term the leader term to start getting changes for. * @param index the log index to start get changes for. - * @param cb the callback to call once the RPC returns. + * @param key the key to start get changes for. + * @param time the time to start get changes for. * @return a deferred object for the response from server. */ - public Deferred getChanges( - HostAndPort hp, YBTable table, String streamId, String tabletId, long term, - long index, Callback cb) { + public Deferred getChangesCDCSDK(YBTable table, String streamId, + String tabletId, long term, + long index, byte[] key, + int write_id, long time) { checkIsClosed(); - TabletClient client = newSimpleClient(hp); - if (client == null) { - throw new IllegalStateException("Could not create a client to " + hp.toString()); - } - GetChangesRequest rpc = new GetChangesRequest(table, streamId, tabletId, term, index); - rpc.setTimeoutMillis(defaultAdminOperationTimeoutMs); - Deferred d = rpc.getDeferred(); - client.sendRpc(rpc); - return d.addCallback(cb); + GetChangesRequest rpc = new GetChangesRequest(table, streamId, tabletId, term, + index, key, write_id, time); + Deferred d = rpc.getDeferred(); + d.addErrback(new Callback() { + @Override + public Exception call(Exception o) throws Exception { + LOG.warn("GetChangesCDCSDK got Errback ", o); + o.printStackTrace(); + throw o; + } + }); + d.addCallback(new Callback() { + @Override + public GetChangesResponse call(GetChangesResponse o) throws Exception { + if (o != null) { + if (o.getResp().hasError()) { + } + } + return o; + } + }); + rpc.setTimeoutMillis(defaultOperationTimeoutMs); + sendRpcToTablet(rpc); + return d; + } + + public Deferred getCheckpoint(YBTable table, + String streamId, String tabletId) { + checkIsClosed(); + GetCheckpointRequest rpc = new GetCheckpointRequest(table, streamId, tabletId); + Deferred d = rpc.getDeferred(); + rpc.setTimeoutMillis(defaultOperationTimeoutMs); + sendRpcToTablet(rpc); + return d; + } + + public Deferred setCheckpoint(YBTable table, + String streamId, String tabletId, + long term, long index) { + checkIsClosed(); + SetCheckpointRequest rpc = new SetCheckpointRequest(table, streamId, tabletId, term, index); + Deferred d = rpc.getDeferred(); + rpc.setTimeoutMillis(defaultOperationTimeoutMs); + sendRpcToTablet(rpc); + return d; } /** @@ -515,7 +591,8 @@ public Deferred deleteTable(final String keyspace, final St * Alter a table on the cluster as specified by the builder. * * When the returned deferred completes it only indicates that the master accepted the alter - * command, use {@link AsyncYBClient#isAlterTableDone(String)} to know when the alter finishes. + * command, use {@link AsyncYBClient#isAlterTableDone(String, String)} to know + * when the alter finishes. * @param keyspace CQL keyspace to which this table belongs * @param name the table's name, if this is a table rename then the old table name must be passed * @param ato the alter table builder @@ -1160,6 +1237,15 @@ public Deferred call(GetTableSchemaResponse response) throws Exception }); } + public Deferred getDBStreamInfo(String streamId) { + checkIsClosed(); + GetDBStreamInfoRequest rpc = new GetDBStreamInfoRequest(this.masterTable, streamId); + Deferred d = rpc.getDeferred(); + rpc.setTimeoutMillis(defaultOperationTimeoutMs); + sendRpcToTablet(rpc); + return d; + } + /** * An RPC that we're never going to send, but can be used to keep track of timeouts and to access * its Deferred. Specifically created for the openTable functions. If the table was just created, @@ -1289,14 +1375,38 @@ Deferred sendRpcToTablet(final YRpc request) { if (cannotRetryRequest(request)) { return tooManyAttemptsOrTimeout(request, null); } + request.attempt++; + final String tableId = request.getTable().getTableId(); byte[] partitionKey = null; + if (request instanceof YRpc.HasKey) { partitionKey = ((YRpc.HasKey)request).partitionKey(); } - final RemoteTablet tablet = getTablet(tableId, partitionKey); - + RemoteTablet tablet = null; + if (partitionKey == null) { + tablet = null; + } + if (isMasterTable(tableId)) { + tablet = getTablet(tableId, partitionKey); + } + if (request instanceof GetChangesRequest) { + String tabletId = ((GetChangesRequest)request).getTabletId(); + tablet = getTablet(tableId, tabletId); + } + if (request instanceof CreateCDCStreamRequest) { + tablet = getFirstTablet(tableId); + } + if (request instanceof GetDBStreamInfoRequest) { + tablet = getFirstTablet(tableId); + } + if (request instanceof GetCheckpointRequest) { + tablet = getFirstTablet(tableId); + } + if (request instanceof SetCheckpointRequest) { + tablet = getFirstTablet(tableId); + } // Set the propagated timestamp so that the next time we send a message to // the server the message includes the last propagated timestamp. long lastPropagatedTs = getLastPropagatedTimestamp(); @@ -1306,6 +1416,7 @@ Deferred sendRpcToTablet(final YRpc request) { if (tablet != null) { TabletClient tabletClient = clientFor(tablet); + if (tabletClient != null) { request.setTablet(tablet); final Deferred d = request.getDeferred(); @@ -1326,8 +1437,10 @@ Deferred sendRpcToTablet(final YRpc request) { } Callback, GetTableLocationsResponsePB> cb = new RetryRpcCB<>(request); Callback, Exception> eb = new RetryRpcErrback<>(request); + Deferred returnedD = locateTablet(request.getTable(), partitionKey); + return AsyncUtil.addCallbacksDeferring(returnedD, cb, eb); } @@ -1337,7 +1450,7 @@ Deferred sendRpcToTablet(final YRpc request) { *

* Use {@code AsyncUtil.addCallbacksDeferring} to add this as the callback and * {@link AsyncYBClient.RetryRpcErrback} as the "errback" to the {@code Deferred} - * returned by {@link #locateTablet(String, byte[])}. + * returned by {@link #locateTablet(YBTable, byte[])}. * @param RPC's return type. * @param Previous query's return type, which we don't use, but need to specify in order to * tie it all together. @@ -1361,8 +1474,8 @@ public String toString() { *

* Use {@code AsyncUtil.addCallbacksDeferring} to add this as the "errback" and * {@link RetryRpcCB} as the callback to the {@code Deferred} returned by - * {@link #locateTablet(String, byte[])}. - * @see #delayedSendRpcToTablet(YRpc, YBException) + * {@link #locateTablet(YBTable, byte[])}. + * @see #delayedSendRpcToTablet(YRpc, YBException, TabletClient) * @param The type of the original RPC. */ final class RetryRpcErrback implements Callback, Exception> { @@ -1372,6 +1485,37 @@ public RetryRpcErrback(YRpc request) { this.request = request; } + @Override + public Deferred call(Exception arg) { + if (arg instanceof NoLeaderMasterFoundException) { + // If we could not find the leader master, try looking up the leader master + // again. + Deferred d = request.getDeferred(); + // TODO: Handle the situation when multiple in-flight RPCs are queued waiting + // for the leader master to be determine (either after a failure or at initialization + // time). This could re-use some of the existing piping in place for non-master tablets. + delayedSendRpcToTablet(request, (NoLeaderMasterFoundException) arg, null); + return d; + } + LOG.info("passing over the exception"); + // Pass all other exceptions through. + this.request.errback(arg); + return Deferred.fromError(arg); + } + + @Override + public String toString() { + return "retry RPC after error"; + } + } + + final class RetryRpcErrbackCDC implements Callback, Exception> { + private final YRpc request; + + public RetryRpcErrbackCDC(YRpc request) { + this.request = request; + } + @Override public Deferred call(Exception arg) { if (arg instanceof NoLeaderMasterFoundException) { @@ -1514,6 +1658,7 @@ long getSleepTimeForRpc(YRpc rpc) { } // TODO backoffs? Sleep in increments of 500 ms, plus some random time up to 50 long sleepTime = (attemptCount * SLEEP_TIME) + sleepRandomizer.nextInt(50); + LOG.info("Going to sleep for " + sleepTime + " at retry " + rpc.attempt); if (LOG.isDebugEnabled()) { LOG.debug("Going to sleep for " + sleepTime + " at retry " + rpc.attempt); } @@ -1560,11 +1705,13 @@ TabletClient clientFor(RemoteTablet tablet) { return null; } if (tablet.leaderIndex == RemoteTablet.NO_LEADER_INDEX) { + LOG.debug("We don't know the leader."); // TODO we don't know where the leader is, either because one wasn't provided or because // we couldn't resolve its IP. We'll just send the client back so it retries and probably // dies after too many attempts. return null; } else { + LOG.debug("We know the leader."); // TODO we currently always hit the leader, we probably don't need to except for writes // and some reads. return tablet.tabletServers.get(tablet.leaderIndex); @@ -1604,6 +1751,8 @@ static Deferred tooManyAttemptsOrTimeout(final YRpc request, return Deferred.fromError(e); } + + /** * Sends a getTableLocations RPC to the master to find the table's tablets. * @param table table to lookup @@ -1620,11 +1769,18 @@ Deferred locateTablet( // this will save us a Master lookup. RemoteTablet tablet = getTablet(tableId, partitionKey); if (tablet != null && clientFor(tablet) != null) { + return Deferred.fromResult(null); // Looks like no lookup needed. } } + + int numTablets = numTabletsInTable; + if (numTabletsInTable != DEFAULT_MAX_TABLETS) { + numTablets = numTabletsInTable; + } GetTableLocationsRequest rpc = - new GetTableLocationsRequest(masterTable, partitionKey, partitionKey, tableId); + new GetTableLocationsRequest(masterTable, partitionKey, partitionKey, tableId, + numTablets); rpc.setTimeoutMillis(defaultAdminOperationTimeoutMs); final Deferred d; @@ -1699,7 +1855,7 @@ private Deferred> loopLocateTable(final String tableId, "Took too long getting the list of tablets, " + deadlineTracker)); } GetTableLocationsRequest rpc = new GetTableLocationsRequest(masterTable, startPartitionKey, - endPartitionKey, tableId); + endPartitionKey, tableId, DEFAULT_MAX_TABLETS); rpc.setTimeoutMillis(defaultAdminOperationTimeoutMs); final Deferred d = sendRpcToTablet(rpc); return d.addCallbackDeferring( @@ -1822,13 +1978,18 @@ private final class MasterLookupCB implements Callback tablets = tabletsCache.get(tableId); + if (tablets == null) { + return null; + } + if (tablets.firstEntry() == null) { + return null; + } + return tablets.firstEntry().getValue(); + + } + + RemoteTablet getTablet(String tableId, String tabletId) { + ConcurrentSkipListMap tablets = tabletsCache.get(tableId); + if (tablets == null) { + return null; + } + // We currently only have one master tablet. + if (isMasterTable(tableId)) { + if (tablets.firstEntry() == null) { + return null; + } + return tablets.firstEntry().getValue(); + } + RemoteTablet rT = this.tablet2client.get(new Slice(tabletId.getBytes())); + return rT; + } + /** * Retrieve the master registration (see {@link GetMasterRegistrationResponse} * for a replica. @@ -2043,7 +2232,6 @@ public void close() throws Exception { * Performs a graceful shutdown of this instance. *

*

    - *
  • {@link AsyncYBSession#flush Flushes} all buffered edits.
  • *
  • Cancels all the other requests.
  • *
  • Terminates all connections.
  • *
  • Releases all other resources.
  • @@ -2774,6 +2962,8 @@ public final static class AsyncYBClientBuilder { private int bossCount = DEFAULT_BOSS_COUNT; private int workerCount = DEFAULT_WORKER_COUNT; + private int numTablets = DEFAULT_MAX_TABLETS; + /** * Creates a new builder for a client that will connect to the specified masters. * @param masterAddresses comma-separated list of "host:port" pairs of the masters @@ -2927,6 +3117,13 @@ public AsyncYBClientBuilder workerCount(int workerCount) { return this; } + public AsyncYBClientBuilder numTablets(int numTablets) { + Preconditions.checkArgument(numTablets > 0, "Number of tablets in a table should " + + "be greater than 0"); + this.numTablets = numTablets; + return this; + } + /** * Creates the channel factory for Netty. The user can specify the executors, but * if they don't, we'll use a simple thread pool. diff --git a/java/yb-client/src/main/java/org/yb/client/CDCErrorException.java b/java/yb-client/src/main/java/org/yb/client/CDCErrorException.java new file mode 100644 index 000000000000..52415bad89a7 --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/CDCErrorException.java @@ -0,0 +1,60 @@ +package org.yb.client; +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// +// The following only applies to changes made to this file as part of YugaByte development. +// +// Portions Copyright (c) YugaByte, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except +// in compliance with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software distributed under the License +// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express +// or implied. See the License for the specific language governing permissions and limitations +// under the License. +// + +import org.yb.annotations.InterfaceAudience; +import org.yb.annotations.InterfaceStability; +import org.yb.cdc.CdcService; +import org.yb.rpc.RpcHeader; + +/** + * This exception is thrown by Tablet Servers when something goes wrong processing a request. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +@SuppressWarnings("serial") +public class CDCErrorException extends YBServerException { + private CdcService.CDCErrorPB tserverError = null; + + CDCErrorException(String serverUuid, CdcService.CDCErrorPB error) { + super(serverUuid, error.getStatus()); + this.tserverError = error; + } + + CDCErrorException(String serverUuid, RpcHeader.ErrorStatusPB errorStatus) { + super(serverUuid, errorStatus); + } + + public CdcService.CDCErrorPB getCDCError() { + return tserverError; + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/CreateCDCStreamRequest.java b/java/yb-client/src/main/java/org/yb/client/CreateCDCStreamRequest.java index 3e416f283a31..0b89ed98574d 100644 --- a/java/yb-client/src/main/java/org/yb/client/CreateCDCStreamRequest.java +++ b/java/yb-client/src/main/java/org/yb/client/CreateCDCStreamRequest.java @@ -1,24 +1,54 @@ package org.yb.client; import com.google.protobuf.Message; +import org.apache.log4j.LogMF; import org.jboss.netty.buffer.ChannelBuffer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.yb.cdc.CdcService.CreateCDCStreamRequestPB; import org.yb.cdc.CdcService.CreateCDCStreamResponsePB; +import org.yb.cdc.CdcService; import org.yb.util.Pair; public class CreateCDCStreamRequest extends YRpc { + public static final Logger LOG = LoggerFactory.getLogger(CreateCDCStreamRequest.class); + private final String tableId; + private final String namespaceName; + private final CdcService.CDCRequestSource source_type; + private final CdcService.CDCRecordFormat record_format; + private final CdcService.CDCCheckpointType checkpoint_type; - public CreateCDCStreamRequest(YBTable masterTable, String tableId) { + public CreateCDCStreamRequest(YBTable masterTable, String tableId, + String namespaceName, String format, + String checkpointType) { super(masterTable); this.tableId = tableId; + this.namespaceName = namespaceName; + this.source_type = CdcService.CDCRequestSource.CDCSDK; + if (format.equalsIgnoreCase("PROTO")) + this.record_format = CdcService.CDCRecordFormat.PROTO; + else { + this.record_format = CdcService.CDCRecordFormat.JSON; + } + if (checkpointType.equalsIgnoreCase("EXPLICIT")) { + this.checkpoint_type = CdcService.CDCCheckpointType.EXPLICIT; + } + else { + this.checkpoint_type = CdcService.CDCCheckpointType.IMPLICIT; + } } @Override ChannelBuffer serialize(Message header) { assert header.isInitialized(); final CreateCDCStreamRequestPB.Builder builder = CreateCDCStreamRequestPB.newBuilder(); - builder.setTableId(this.tableId); + if (namespaceName == null) + builder.setTableId(this.tableId); + builder.setNamespaceName(this.namespaceName); + builder.setSourceType(this.source_type); + builder.setRecordFormat(this.record_format); + builder.setCheckpointType(this.checkpoint_type); return toChannelBuffer(header, builder.build()); } @@ -37,7 +67,7 @@ Pair deserialize( readProtobuf(callResponse.getPBMessage(), respBuilder); CreateCDCStreamResponse response = new CreateCDCStreamResponse( - deadlineTracker.getElapsedMillis(), uuid, respBuilder.getStreamId().toStringUtf8()); + deadlineTracker.getElapsedMillis(), uuid, respBuilder.getDbStreamId().toStringUtf8()); return new Pair( response, respBuilder.hasError() ? respBuilder.getError() : null); } diff --git a/java/yb-client/src/main/java/org/yb/client/GetChangesRequest.java b/java/yb-client/src/main/java/org/yb/client/GetChangesRequest.java index 0da934a2527b..8a11c16724b4 100644 --- a/java/yb-client/src/main/java/org/yb/client/GetChangesRequest.java +++ b/java/yb-client/src/main/java/org/yb/client/GetChangesRequest.java @@ -24,29 +24,43 @@ public class GetChangesRequest extends YRpc { private final String streamId; + + public String getTabletId() { + return tabletId; + } + private final String tabletId; private final long term; private final long index; + private final byte[] key; + private final int write_id; + private final long time; - public GetChangesRequest(YBTable table, String streamId, String tabletId, long term, long index) { + public GetChangesRequest(YBTable table, String streamId, String tabletId, + long term, long index, byte[] key, int write_id, long time) { super(table); this.streamId = streamId; this.tabletId = tabletId; this.term = term; this.index = index; + this.key = key; + this.write_id = write_id; + this.time = time; } @Override ChannelBuffer serialize(Message header) { assert header.isInitialized(); final GetChangesRequestPB.Builder builder = GetChangesRequestPB.newBuilder(); - builder.setStreamId(ByteString.copyFromUtf8(this.streamId)); + builder.setDbStreamId(ByteString.copyFromUtf8(this.streamId)); builder.setTabletId(ByteString.copyFromUtf8(this.tabletId)); if (term != 0 || index != 0) { - CdcService.CDCCheckpointPB.Builder checkpointBuilder = - CdcService.CDCCheckpointPB.newBuilder(); - checkpointBuilder.setOpId(Opid.OpIdPB.newBuilder().setIndex(this.index).setTerm(this.term)); - builder.setFromCheckpoint(checkpointBuilder); + CdcService.CDCSDKCheckpointPB.Builder checkpointBuilder = + CdcService.CDCSDKCheckpointPB.newBuilder(); + checkpointBuilder.setIndex(this.index).setTerm(this.term) + .setKey(ByteString.copyFrom(this.key)).setWriteId(this.write_id) + .setSnapshotTime(this.time); + builder.setFromCdcSdkCheckpoint(checkpointBuilder.build()); } return toChannelBuffer(header, builder.build()); } @@ -65,7 +79,9 @@ Pair deserialize( final GetChangesResponsePB.Builder respBuilder = GetChangesResponsePB.newBuilder(); readProtobuf(callResponse.getPBMessage(), respBuilder); GetChangesResponse response = new GetChangesResponse( - deadlineTracker.getElapsedMillis(), uuid, respBuilder.build()); + deadlineTracker.getElapsedMillis(), uuid, respBuilder.build(), + respBuilder.getCdcSdkCheckpointBuilder().getKey().toByteArray(), + respBuilder.getCdcSdkCheckpointBuilder().getWriteId()); return new Pair( response, respBuilder.hasError() ? respBuilder.getError() : null); } diff --git a/java/yb-client/src/main/java/org/yb/client/GetChangesResponse.java b/java/yb-client/src/main/java/org/yb/client/GetChangesResponse.java index a3dc38d66ad1..2250504dc64d 100644 --- a/java/yb-client/src/main/java/org/yb/client/GetChangesResponse.java +++ b/java/yb-client/src/main/java/org/yb/client/GetChangesResponse.java @@ -14,18 +14,47 @@ package org.yb.client; import org.yb.annotations.InterfaceAudience; +import org.yb.cdc.CdcService; import org.yb.cdc.CdcService.GetChangesResponsePB; +import org.yb.util.Pair; @InterfaceAudience.Public public class GetChangesResponse extends YRpcResponse { private final GetChangesResponsePB resp; - GetChangesResponse(long ellapsedMillis, String uuid, GetChangesResponsePB resp) { + private byte[] key; + private int writeId; + + GetChangesResponse(long ellapsedMillis, String uuid, + GetChangesResponsePB resp, byte[] key, int writeId) { super(ellapsedMillis, uuid); + this.key = key; + this.writeId = writeId; this.resp = resp; } public GetChangesResponsePB getResp() { return resp; } + + public byte[] getKey() { + return key; + } + + public int getWriteId() { + return writeId; + } + + public long getTerm() { + return getResp().getCdcSdkCheckpoint().getTerm(); + } + + public long getIndex() { + return getResp().getCdcSdkCheckpoint().getIndex(); + } + + public long getSnapshotTime() { + return getResp().getCdcSdkCheckpoint().getSnapshotTime(); + } + } diff --git a/java/yb-client/src/main/java/org/yb/client/GetCheckpointRequest.java b/java/yb-client/src/main/java/org/yb/client/GetCheckpointRequest.java new file mode 100644 index 000000000000..a5c2fdc75577 --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/GetCheckpointRequest.java @@ -0,0 +1,52 @@ +package org.yb.client; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Message; +import org.jboss.netty.buffer.ChannelBuffer; +import org.yb.cdc.CdcService; +import org.yb.util.Pair; + +public class GetCheckpointRequest extends YRpc{ + private String streamId; + private String tabletId; + + public GetCheckpointRequest(YBTable table, String streamId, String tabletId) { + super(table); + this.streamId = streamId; + this.tabletId = tabletId; + } + + @Override + ChannelBuffer serialize(Message header) { + assert header.isInitialized(); + final CdcService.GetCheckpointRequestPB.Builder builder = CdcService + .GetCheckpointRequestPB.newBuilder(); + builder.setStreamId(ByteString.copyFromUtf8(this.streamId)); + builder.setTabletId(ByteString.copyFromUtf8(this.tabletId)); + + return toChannelBuffer(header, builder.build()); + } + + @Override + String serviceName() { return CDC_SERVICE_NAME; } + + @Override + String method() { + return "GetCheckpoint"; + } + + @Override + Pair deserialize(CallResponse callResponse, + String uuid) throws Exception { + final CdcService.GetCheckpointResponsePB.Builder respBuilder = + CdcService.GetCheckpointResponsePB.newBuilder(); + readProtobuf(callResponse.getPBMessage(), respBuilder); + + GetCheckpointResponse response = + new GetCheckpointResponse(deadlineTracker.getElapsedMillis(), uuid, + respBuilder.getCheckpoint().getOpId().getIndex(), + respBuilder.getCheckpoint().getOpId().getTerm()); + return new Pair( + response, respBuilder.hasError() ? respBuilder.getError() : null); + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/GetCheckpointResponse.java b/java/yb-client/src/main/java/org/yb/client/GetCheckpointResponse.java new file mode 100644 index 000000000000..9a2764ff88ac --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/GetCheckpointResponse.java @@ -0,0 +1,21 @@ +package org.yb.client; + +public class GetCheckpointResponse extends YRpcResponse { + + private long index; + private long term; + + public GetCheckpointResponse(long elapsedMillis, String uuid, long index, long term) { + super(elapsedMillis, uuid); + this.index = index; + this.term = term; + } + + public long getIndex() { + return index; + } + + public long getTerm() { + return term; + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoRequest.java b/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoRequest.java new file mode 100644 index 000000000000..238dc3be120f --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoRequest.java @@ -0,0 +1,49 @@ +package org.yb.client; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Message; +import org.jboss.netty.buffer.ChannelBuffer; +import org.yb.master.MasterReplicationOuterClass; +import org.yb.util.Pair; + +public class GetDBStreamInfoRequest extends YRpc{ + + private final String streamId; + + GetDBStreamInfoRequest(YBTable table, String streamId) { + super(table); + this.streamId = streamId; + } + + @Override + ChannelBuffer serialize(Message header) { + assert header.isInitialized(); + final MasterReplicationOuterClass.GetCDCDBStreamInfoRequestPB.Builder builder = + MasterReplicationOuterClass.GetCDCDBStreamInfoRequestPB.newBuilder(); + builder.setDbStreamId(ByteString.copyFromUtf8(this.streamId)); + return toChannelBuffer(header, builder.build()); + } + + @Override + String serviceName() { return MASTER_SERVICE_NAME; } + + @Override + String method() { + return "GetCDCDBStreamInfo"; + } + + @Override + Pair deserialize(CallResponse callResponse, + String tsUUID) throws Exception { + final MasterReplicationOuterClass.GetCDCDBStreamInfoResponsePB.Builder respBuilder = + MasterReplicationOuterClass.GetCDCDBStreamInfoResponsePB.newBuilder(); + readProtobuf(callResponse.getPBMessage(), respBuilder); + + + GetDBStreamInfoResponse response = + new GetDBStreamInfoResponse(deadlineTracker.getElapsedMillis(), tsUUID, + respBuilder.getTableInfoList()); + return new Pair( + response, respBuilder.hasError() ? respBuilder.getError() : null); + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoResponse.java b/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoResponse.java new file mode 100644 index 000000000000..0ee1c5044642 --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/GetDBStreamInfoResponse.java @@ -0,0 +1,24 @@ +package org.yb.client; + +import org.yb.master.MasterReplicationOuterClass; + +import java.util.List; + +public class GetDBStreamInfoResponse extends YRpcResponse { + + private final List + tableInfoList; + + public GetDBStreamInfoResponse(long elapsedMillis, String tsUUID, + List + tableInfoList) { + super(elapsedMillis, tsUUID); + this.tableInfoList = tableInfoList; + } + + public List getTableInfoList() { + return tableInfoList; + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/GetMasterRegistrationReceived.java b/java/yb-client/src/main/java/org/yb/client/GetMasterRegistrationReceived.java index c46ea121c034..790ca65608e9 100644 --- a/java/yb-client/src/main/java/org/yb/client/GetMasterRegistrationReceived.java +++ b/java/yb-client/src/main/java/org/yb/client/GetMasterRegistrationReceived.java @@ -123,7 +123,6 @@ private void incrementCountAndCheckExhausted() { if (countResponsesReceived.incrementAndGet() == numMasters) { if (responseDCalled.compareAndSet(false, true)) { boolean allUnrecoverable = true; - // When there are no exceptions, default to retry semantics. if (exceptionsReceived.isEmpty()) { allUnrecoverable = false; @@ -135,6 +134,12 @@ private void incrementCountAndCheckExhausted() { break; } } + for (Exception ex : exceptionsReceived) { + if (!(ex instanceof NoLeaderMasterFoundException)) { + allUnrecoverable = false; + break; + } + } String allHosts = NetUtil.hostsAndPortsToString(masterAddrs); // Doing a negative check because allUnrecoverable stays true if there are no exceptions. if (!allUnrecoverable) { @@ -230,7 +235,6 @@ public GetMasterRegistrationErrCB(HostAndPort hostAndPort) { @Override public Void call(Exception e) throws Exception { - LOG.warn("Error receiving a response from: " + hostAndPort); exceptionsReceived.add(e); incrementCountAndCheckExhausted(); return null; diff --git a/java/yb-client/src/main/java/org/yb/client/GetTableLocationsRequest.java b/java/yb-client/src/main/java/org/yb/client/GetTableLocationsRequest.java index e70ded7ca00c..5080a98e1abb 100644 --- a/java/yb-client/src/main/java/org/yb/client/GetTableLocationsRequest.java +++ b/java/yb-client/src/main/java/org/yb/client/GetTableLocationsRequest.java @@ -49,9 +49,10 @@ class GetTableLocationsRequest extends YRpc 0) { @@ -61,6 +62,7 @@ class GetTableLocationsRequest extends YRpc{ + private String streamId; + private String tabletId; + private long index; + private long term; + + public SetCheckpointRequest(YBTable table, String streamId, + String tabletId, long term, long index) { + super(table); + this.streamId = streamId; + this.tabletId = tabletId; + this.term = term; + this.index = index; + } + + @Override + ChannelBuffer serialize(Message header) { + assert header.isInitialized(); + final CdcService.SetCDCCheckpointRequestPB.Builder builder = CdcService + .SetCDCCheckpointRequestPB.newBuilder(); + builder.setStreamId(ByteString.copyFromUtf8(this.streamId)); + builder.setTabletId(ByteString.copyFromUtf8(this.tabletId)); + final CdcService.CDCCheckpointPB.Builder cBuilder = CdcService + .CDCCheckpointPB.newBuilder(); + builder.setCheckpoint(cBuilder.setOpId(Opid.OpIdPB.newBuilder().setIndex(this.index) + .setTerm(this.term).build()).build()); + return toChannelBuffer(header, builder.build()); + } + + @Override + String serviceName() { return CDC_SERVICE_NAME; } + + @Override + String method() { + return "SetCDCCheckpoint"; + } + + @Override + Pair deserialize(CallResponse callResponse, + String uuid) throws Exception { + final CdcService.SetCDCCheckpointResponsePB.Builder respBuilder = + CdcService.SetCDCCheckpointResponsePB.newBuilder(); + readProtobuf(callResponse.getPBMessage(), respBuilder); + SetCheckpointResponse response = + new SetCheckpointResponse(deadlineTracker.getElapsedMillis(), uuid); + return new Pair( + response, respBuilder.hasError() ? respBuilder.getError() : null); + } +} diff --git a/java/yb-client/src/main/java/org/yb/client/SetCheckpointResponse.java b/java/yb-client/src/main/java/org/yb/client/SetCheckpointResponse.java new file mode 100644 index 000000000000..f158ab2627af --- /dev/null +++ b/java/yb-client/src/main/java/org/yb/client/SetCheckpointResponse.java @@ -0,0 +1,8 @@ +package org.yb.client; + +public class SetCheckpointResponse extends YRpcResponse { + public SetCheckpointResponse(long elapsedMillis, String tsUUID) { + super(elapsedMillis, tsUUID); + } + +} diff --git a/java/yb-client/src/main/java/org/yb/client/TabletClient.java b/java/yb-client/src/main/java/org/yb/client/TabletClient.java index 5ecc50a352d6..c63e4939776a 100644 --- a/java/yb-client/src/main/java/org/yb/client/TabletClient.java +++ b/java/yb-client/src/main/java/org/yb/client/TabletClient.java @@ -47,6 +47,7 @@ import org.jboss.netty.handler.timeout.ReadTimeoutException; import org.yb.WireProtocol; import org.yb.annotations.InterfaceAudience; +import org.yb.cdc.CdcService; import org.yb.master.MasterTypes; import org.yb.rpc.RpcHeader; import org.yb.tserver.Tserver; @@ -423,7 +424,6 @@ protected Object decode(ChannelHandlerContext ctx, Channel chan, ChannelBuffer b // We're going to errback. decoded = null; } - } else if (decoded.getSecond() instanceof MasterTypes.MasterErrorPB) { MasterTypes.MasterErrorPB error = (MasterTypes.MasterErrorPB) decoded.getSecond(); exception = dispatchMasterErrorOrReturnException(rpc, error); @@ -434,6 +434,17 @@ protected Object decode(ChannelHandlerContext ctx, Channel chan, ChannelBuffer b decoded = null; } } + else if (decoded.getSecond() instanceof CdcService.CDCErrorPB) { + CdcService.CDCErrorPB error = (CdcService.CDCErrorPB) decoded.getSecond(); + exception = dispatchCDCErrorOrReturnException(rpc, error); + if (exception == null) { + // It was taken care of. + return null; + } else { + // We're going to errback. + decoded = null; + } + } } try { @@ -491,6 +502,36 @@ private Exception dispatchTSErrorOrReturnException(YRpc rpc, return null; } + private Exception dispatchCDCErrorOrReturnException(YRpc rpc, + CdcService.CDCErrorPB error) { + WireProtocol.AppStatusPB.ErrorCode code = error.getStatus().getCode(); + CDCErrorException ex = new CDCErrorException(uuid, error); + if (error.getCode() == CdcService.CDCErrorPB.Code.TABLET_NOT_FOUND) { + // rpc.deadlineTracker.reset(); + ybClient.handleNotLeader(rpc, ex, this); + //ybClient.handleTabletNotFound(rpc, ex, this); + // we're not calling rpc.callback() so we rely on the client to retry that RPC + } else if (code == WireProtocol.AppStatusPB.ErrorCode.SERVICE_UNAVAILABLE /*|| + code == WireProtocol.AppStatusPB.ErrorCode.LEADER_NOT_READY_TO_SERVE || + error.getCode() == + Tserver.TabletServerErrorPB.Code.LEADER_NOT_READY_CHANGE_CONFIG || + error.getCode() == + Tserver.TabletServerErrorPB.Code.LEADER_NOT_READY_TO_STEP_DOWN || + error.getCode() == + Tserver.TabletServerErrorPB.Code.LEADER_NOT_READY_TO_SERVE*/) { + ybClient.handleRetryableError(rpc, ex, this); + // The following error codes are an indication that the tablet isn't a leader, or, in case + // of LEADER_HAS_NO_LEASE, might no longer be the leader due to failing to replicate a leader + // lease, so we retry looking up the leader anyway. + } else if (code == WireProtocol.AppStatusPB.ErrorCode.LEADER_HAS_NO_LEASE || + code == WireProtocol.AppStatusPB.ErrorCode.ILLEGAL_STATE || + code == WireProtocol.AppStatusPB.ErrorCode.ABORTED || + error.getCode() == CdcService.CDCErrorPB.Code.NOT_LEADER) { + ybClient.handleNotLeader(rpc, ex, this); + } + return ex; + } + /** * Provides different handling for various kinds of master errors: re-uses the * mechanisms already in place for handling tablet server errors as much as possible. diff --git a/java/yb-client/src/main/java/org/yb/client/YBClient.java b/java/yb-client/src/main/java/org/yb/client/YBClient.java index f03414d7a98e..bd8ae13fe023 100644 --- a/java/yb-client/src/main/java/org/yb/client/YBClient.java +++ b/java/yb-client/src/main/java/org/yb/client/YBClient.java @@ -1222,8 +1222,23 @@ public ListTablesResponse getTablesList( * @return a deferred object for the response from server. */ public CreateCDCStreamResponse createCDCStream( - final HostAndPort hp, String tableId) throws Exception{ - Deferred d = asyncClient.createCDCStream(hp, tableId); + final HostAndPort hp, String tableId, + String nameSpaceName, String format, + String checkpointType) throws Exception{ + Deferred d = asyncClient.createCDCStream(hp, + tableId, + nameSpaceName, + format, + checkpointType); + return d.join(getDefaultAdminOperationTimeoutMs()); + } + + public CreateCDCStreamResponse createCDCStream(YBTable table, + String nameSpaceName, + String format, + String checkpointType) throws Exception { + Deferred d = asyncClient.createCDCStream(table, + nameSpaceName, format, checkpointType); return d.join(getDefaultAdminOperationTimeoutMs()); } @@ -1399,6 +1414,47 @@ public IsSetupUniverseReplicationDoneResponse isAlterUniverseReplicationDone( return d.join(getDefaultAdminOperationTimeoutMs()); } + public GetChangesResponse getChangesCDCSDK(YBTable table, String streamId, + String tabletId, long term, + long index, byte[] key, + int write_id, long time) throws Exception { + Deferred d = asyncClient + .getChangesCDCSDK(table, streamId, tabletId, term, index, key, write_id, time); + return d.join(2*getDefaultAdminOperationTimeoutMs()); + } + + public GetCheckpointResponse getCheckpoint(YBTable table, String streamId, + String tabletId) throws Exception { + Deferred d = asyncClient + .getCheckpoint(table, streamId, tabletId); + return d.join(2*getDefaultAdminOperationTimeoutMs()); + } + + public GetDBStreamInfoResponse getDBStreamInfo(String streamId) throws Exception { + Deferred d = asyncClient + .getDBStreamInfo(streamId); + return d.join(2*getDefaultAdminOperationTimeoutMs()); + } + + public SetCheckpointResponse commitCheckpoint(YBTable table, String streamId, + String tabletId, + long term, + long index) throws Exception { + Deferred d = asyncClient + .setCheckpoint(table, streamId, tabletId, term, index); + d.addErrback(new Callback() { + @Override + public Exception call(Exception o) throws Exception { + o.printStackTrace(); + throw o; + } + }); + d.addCallback(setCheckpointResponse -> { + return setCheckpointResponse; + }); + return d.join(2 * getDefaultAdminOperationTimeoutMs()); + } + public DeleteUniverseReplicationResponse deleteUniverseReplication( String replicationGroupName) throws Exception { Deferred d = @@ -1406,6 +1462,7 @@ public DeleteUniverseReplicationResponse deleteUniverseReplication( return d.join(getDefaultAdminOperationTimeoutMs()); } + public GetUniverseReplicationResponse getUniverseReplication( String replicationGrouopName) throws Exception { Deferred d = diff --git a/java/yb-client/src/main/java/org/yb/client/YRpc.java b/java/yb-client/src/main/java/org/yb/client/YRpc.java index aab3851e8fef..4c7f294b734c 100644 --- a/java/yb-client/src/main/java/org/yb/client/YRpc.java +++ b/java/yb-client/src/main/java/org/yb/client/YRpc.java @@ -44,6 +44,8 @@ import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; import com.stumbleupon.async.Deferred; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.yb.annotations.InterfaceAudience; import org.yb.util.Pair; import org.yb.util.Slice; @@ -69,6 +71,7 @@ */ @InterfaceAudience.Private public abstract class YRpc { + public static final Logger LOG = LoggerFactory.getLogger(YRpc.class); // Service names. protected static final String GENERIC_SERVICE_NAME = "yb.server.GenericService"; diff --git a/java/yb-client/src/test/java/org/yb/client/BaseYBClientTest.java b/java/yb-client/src/test/java/org/yb/client/BaseYBClientTest.java index bb9c6599d9ac..0e16b3a618df 100644 --- a/java/yb-client/src/test/java/org/yb/client/BaseYBClientTest.java +++ b/java/yb-client/src/test/java/org/yb/client/BaseYBClientTest.java @@ -236,9 +236,36 @@ protected static void killTabletLeader(YBTable table) throws Exception { } HostAndPort leaderHostPort = HostAndPort.fromParts(leader.getRpcHost(), leader.getRpcPort()); + LOG.info("The previous host port is " + leaderHostPort); miniCluster.killTabletServerOnHostPort(leaderHostPort); } + protected static void killAllTabletLeader(YBTable table) throws Exception { + List tablets = table.getTabletsLocations(DEFAULT_SLEEP); + for (LocatedTablet tablet : tablets) { + LocatedTablet.Replica leader = null; + DeadlineTracker deadlineTracker = new DeadlineTracker(); + deadlineTracker.setDeadline(DEFAULT_SLEEP); + while (leader == null) { + if (deadlineTracker.timedOut()) { + fail("Timed out while trying to find a leader for this table: " + table.getName()); + } + + if (tablet.getReplicas().size() == 1) { + fail("Table " + table.getName() + " only has 1 tablet, please enable replication"); + } + leader = tablet.getLeaderReplica(); + if (leader == null) { + LOG.info("Sleeping while waiting for a tablet LEADER to arise, currently slept " + + deadlineTracker.getElapsedMillis() + "ms"); + Thread.sleep(50); + } + } + HostAndPort leaderHostPort = HostAndPort.fromParts(leader.getRpcHost(), leader.getRpcPort()); + miniCluster.killTabletServerOnHostPort(leaderHostPort); + } + } + /** * Find the host/port of the leader master. * @return The host/port of the leader master. diff --git a/java/yb-client/src/test/java/org/yb/client/TestAsyncYBClient.java b/java/yb-client/src/test/java/org/yb/client/TestAsyncYBClient.java index a94957c41d68..9b2a0ce029fd 100644 --- a/java/yb-client/src/test/java/org/yb/client/TestAsyncYBClient.java +++ b/java/yb-client/src/test/java/org/yb/client/TestAsyncYBClient.java @@ -34,6 +34,7 @@ import com.google.common.base.Charsets; import com.google.protobuf.ByteString; import com.stumbleupon.async.Deferred; +import com.stumbleupon.async.TimeoutException; import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; @@ -64,6 +65,7 @@ protected void afterStartingMiniCluster() throws Exception { @Test public void testBadHostnames() throws Exception { String badHostname = "some-unknown-host-hopefully"; + String timedOutExceptionMessage = "Timed out after 1000ms"; // Test that a bad hostname for the master makes us error out quickly. AsyncYBClient invalidClient = new AsyncYBClient.AsyncYBClientBuilder(badHostname).build(); @@ -71,8 +73,8 @@ public void testBadHostnames() throws Exception { invalidClient.listTabletServers().join(1000); fail("This should have failed quickly"); } catch (Exception ex) { - assertTrue(ex instanceof NonRecoverableException); - assertTrue(ex.getMessage().contains(badHostname)); + assertTrue(ex instanceof TimeoutException); + assertTrue(ex.getMessage().contains(timedOutExceptionMessage)); } MasterClientOuterClass.GetTableLocationsResponsePB.Builder builder = diff --git a/java/yb-client/src/test/java/org/yb/minicluster/BaseMiniClusterTest.java b/java/yb-client/src/test/java/org/yb/minicluster/BaseMiniClusterTest.java index 2cfc55bbb02d..932e945f91dc 100644 --- a/java/yb-client/src/test/java/org/yb/minicluster/BaseMiniClusterTest.java +++ b/java/yb-client/src/test/java/org/yb/minicluster/BaseMiniClusterTest.java @@ -127,9 +127,6 @@ protected Map getMasterFlags() { flagMap.put("durable_wal_write", "false"); - // Limit number of transaction table tablets to help avoid timeouts. - flagMap.put("transaction_table_num_tablets", Integer.toString(NUM_TABLET_SERVERS)); - // For sanitizer builds, it is easy to overload the master, leading to quorum changes. // This could end up breaking ever trivial DDLs like creating an initial table in the cluster. if (BuildTypeUtil.isSanitizerBuild()) { diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java index 863bd0d88c33..a6784b4fc420 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java @@ -88,10 +88,8 @@ public class BasePgSQLTest extends BaseMiniClusterTest { protected static final String COMMIT_STMT_METRIC = METRIC_PREFIX + "CommitStmt"; protected static final String ROLLBACK_STMT_METRIC = METRIC_PREFIX + "RollbackStmt"; protected static final String OTHER_STMT_METRIC = METRIC_PREFIX + "OtherStmts"; - protected static final String SINGLE_SHARD_TRANSACTIONS_METRIC_DEPRECATED = METRIC_PREFIX - + "Single_Shard_Transactions"; protected static final String SINGLE_SHARD_TRANSACTIONS_METRIC = - METRIC_PREFIX + "SingleShardTransactions"; + METRIC_PREFIX + "Single_Shard_Transactions"; protected static final String TRANSACTIONS_METRIC = METRIC_PREFIX + "Transactions"; protected static final String AGGREGATE_PUSHDOWNS_METRIC = METRIC_PREFIX + "AggregatePushdowns"; protected static final String CATALOG_CACHE_MISSES_METRICS = METRIC_PREFIX + "CatalogCacheMisses"; @@ -780,24 +778,18 @@ protected long verifyStatementMetric( Statement statement, String query, String metricName, int queryMetricDelta, int singleShardTxnMetricDelta, int txnMetricDelta, boolean validStmt) throws Exception { return verifyQuery( - statement, query, validStmt, - new MetricCountChecker( - SINGLE_SHARD_TRANSACTIONS_METRIC_DEPRECATED, this::getMetric, - singleShardTxnMetricDelta), - new MetricCountChecker( - SINGLE_SHARD_TRANSACTIONS_METRIC, this::getMetric, singleShardTxnMetricDelta), - new MetricCountChecker(TRANSACTIONS_METRIC, this::getMetric, txnMetricDelta), - new MetricCountChecker(metricName, this::getMetric, queryMetricDelta)); + statement, query, validStmt, + new MetricCountChecker( + SINGLE_SHARD_TRANSACTIONS_METRIC, this::getMetric, singleShardTxnMetricDelta), + new MetricCountChecker(TRANSACTIONS_METRIC, this::getMetric, txnMetricDelta), + new MetricCountChecker(metricName, this::getMetric, queryMetricDelta)); } protected void verifyStatementTxnMetric( Statement statement, String query, int singleShardTxnMetricDelta) throws Exception { verifyQuery( statement, query,true, - new MetricCountChecker( - SINGLE_SHARD_TRANSACTIONS_METRIC_DEPRECATED, this::getMetric, - singleShardTxnMetricDelta), - new MetricCountChecker( + new MetricCountChecker( SINGLE_SHARD_TRANSACTIONS_METRIC, this::getMetric, singleShardTxnMetricDelta)); } diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAlterTableAddPrimaryKey.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAlterTableAddPrimaryKey.java index da90d48d0957..13521b25b4ed 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAlterTableAddPrimaryKey.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAlterTableAddPrimaryKey.java @@ -48,15 +48,6 @@ public void simplest() throws Exception { } } - @Test - public void withNoForceRowLevelSecurity() throws Exception { - try (Statement stmt = connection.createStatement()) { - stmt.executeUpdate("CREATE TABLE nopk (id int)"); - alterAddPrimaryKey(stmt, "nopk", "NO FORCE ROW LEVEL SECURITY, " + - "ADD PRIMARY KEY (id)", 1, NUM_TABLET_SERVERS); - } - } - @Test public void withFillFactor() throws Exception { try (Statement stmt = connection.createStatement()) { diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgReadTimeout.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgReadTimeout.java index 55c43972c86d..f1446873e805 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgReadTimeout.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgReadTimeout.java @@ -22,9 +22,7 @@ import java.sql.ResultSet; import java.sql.Statement; import java.util.Map; -import java.util.regex.Pattern; -import com.yugabyte.util.PSQLException; import static org.yb.AssertionWrappers.*; @RunWith(value=YBTestRunnerNonTsanOnly.class) @@ -37,8 +35,6 @@ protected Map getTServerFlags() { Map flagMap = super.getTServerFlags(); // Limit Read RPC timeout to reduce test running time flagMap.put("pg_yb_session_timeout_ms", Long.toString(kPgYbSessionTimeoutMs)); - // Verbose logging to help investigating if test fails - flagMap.put("vmodule", "pgsql_operation=1"); return flagMap; } @@ -60,15 +56,13 @@ protected int getInitialNumTServers() { } /** - * Test if a long query can run without triggering Read RPC timeout. + * Test if long aggregate query can run without triggering Read RPC timeout. + * Aggregate queries may scan entire partition without returning any results. + * If there are too many rows, scan may take too long, and connection may + * timeout. To prevent that DocDB returns partial result when deadline is + * approaching, and client sends another request to resume the scan. * - * Regular scan reaches the per response row limit pretty soon, well before - * the session timeout, but scans with aggregates return single row after - * scan completion, so they may timeout on large tables. To prevent that - * DocDB returns partial result when deadline is approaching, and client - * sends another request to resume the scan. - * - * This test loads chunks of data into a table, until "SELECT count(*)" + * This test loads chuncks of data into a table, until "SELECT count(*)" * against this table takes at least twice as long as the timeout to make sure * that partial result is returned in time and scan is properly resumed. * @@ -87,7 +81,6 @@ public void testReadTimeout() throws Exception { int chunk_size = 10000; int rows_loaded = 0; while (true) { - ResultSet rs; // generate chunk_size rows int from_val = rows_loaded + 1; int to_val = rows_loaded + chunk_size; @@ -96,30 +89,16 @@ public void testReadTimeout() throws Exception { LOG.info("Loaded " + rows_loaded + " rows"); // count the rows final long startTimeMillis = System.currentTimeMillis(); - try { - rs = statement.executeQuery(query); - } catch (PSQLException ex) { - if (Pattern.matches(".*RPC .* timed out after.*", ex.getMessage())) { - throw new Exception("Please check GitHub issue #11477", ex); - } - throw ex; - } + ResultSet rs = statement.executeQuery(query); long durationMillis = System.currentTimeMillis() - startTimeMillis; LOG.info("SELECT count(*) FROM readtimeouttest; took " + durationMillis + "ms"); - // if select statement took long enough check result and exit, + // if se;ect statement took long enough check result and exit, // otherwise generate more rows and try again if (durationMillis > 2 * kPgYbSessionTimeoutMs) { assertTrue(rs.next()); assertEquals(rows_loaded, rs.getInt(1)); break; } - // Adjust number of rows to load to achieve desired query duration next time. - // Adding extra 20%, for random factors that may affect the duration. - // If next time duration don't make it, we'll need another try. - double coefficient = 2.4 * kPgYbSessionTimeoutMs / durationMillis - 1; - // Do at least 10000 rows per load to avoid situation when we repeatedly - // load few rows, but duration is a bit shy of double timeout. - chunk_size = Math.max((int) (coefficient * rows_loaded), 10000); } LOG.info("Done with the test"); } diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressPushdown.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressPushdown.java deleted file mode 100644 index 31f5725b5fd7..000000000000 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressPushdown.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) YugaByte, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except -// in compliance with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software distributed under the License -// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express -// or implied. See the License for the specific language governing permissions and limitations -// under the License. -// -package org.yb.pgsql; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.yb.util.YBTestRunnerNonTsanOnly; - -/** - * Runs the pg_regress test suite on YB code. - */ -@RunWith(value=YBTestRunnerNonTsanOnly.class) -public class TestPgRegressPushdown extends BasePgSQLTest { - @Override - public int getTestMethodTimeoutSec() { - return 1800; - } - - @Test - public void testPgRegressPushdown() throws Exception { - runPgRegressTest("yb_pg_pushdown_serial_schedule"); - } -} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestTablespaceProperties.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestTablespaceProperties.java index a1a75ceb382e..965232758ff2 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestTablespaceProperties.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestTablespaceProperties.java @@ -57,6 +57,8 @@ public class TestTablespaceProperties extends BasePgSQLTest { private static final int MASTER_LOAD_BALANCER_WAIT_TIME_MS = 60 * 1000; + private static final int TRANSACTION_TABLE_NUM_TABLETS = 4; + private static final int LOAD_BALANCER_MAX_CONCURRENT = 10; private static final String tablespaceName = "testTablespace"; @@ -90,6 +92,11 @@ protected void customizeMiniClusterBuilder(MiniYBClusterBuilder builder) { builder.addMasterFlag("auto_create_local_transaction_tables", "true"); builder.addMasterFlag("TEST_name_transaction_tables_with_tablespace_id", "true"); + // Default behavior is to scale based on number of CPU cores, which will make + // load balancing transaction tables too much time and time out tests. + builder.addMasterFlag("transaction_table_num_tablets", + Integer.toString(TRANSACTION_TABLE_NUM_TABLETS)); + // We wait for the load balancer whenever it gets triggered anyways, so there's // no concerns about the load balancer taking too many resources. builder.addMasterFlag("load_balancer_max_concurrent_tablet_remote_bootstraps", @@ -111,7 +118,6 @@ protected void customizeMiniClusterBuilder(MiniYBClusterBuilder builder) { @Before public void setupTablespaces() throws Exception { try (Statement setupStatement = connection.createStatement()) { - setupStatement.execute("DROP TABLESPACE IF EXISTS " + tablespaceName); setupStatement.execute( " CREATE TABLESPACE " + tablespaceName + " WITH (replica_placement=" + @@ -202,36 +208,6 @@ public void testTablespaces() throws Exception { testLBTablespacePlacement(); } - @Test - public void testTablesOptOutOfColocation() throws Exception { - final String dbname = "testdatabase"; - try (Statement stmt = connection.createStatement()) { - stmt.execute(String.format("CREATE DATABASE %s COLOCATED=TRUE", dbname)); - } - final String colocatedTableName = "colocated_table"; - final String nonColocatedTable = "colocation_opt_out_table"; - try (Connection connection2 = getConnectionBuilder().withDatabase(dbname).connect(); - Statement stmt = connection2.createStatement()) { - stmt.execute(String.format("CREATE TABLE %s (h INT PRIMARY KEY, a INT, b FLOAT) " + - "WITH (colocated = false) TABLESPACE testTablespace", - nonColocatedTable)); - stmt.execute(String.format("CREATE TABLE %s (h INT PRIMARY KEY, a INT, b FLOAT)", - colocatedTableName)); - } - verifyDefaultPlacement(colocatedTableName); - verifyCustomPlacement(nonColocatedTable); - - // Wait for tablespace info to be refreshed in load balancer. - Thread.sleep(5 * MASTER_REFRESH_TABLESPACE_INFO_SECS); - - // Verify that load balancer is indeed idle. - assertTrue(miniCluster.getClient().waitForLoadBalancerIdle( - MASTER_LOAD_BALANCER_WAIT_TIME_MS)); - - verifyDefaultPlacement(colocatedTableName); - verifyCustomPlacement(nonColocatedTable); - } - public void executeAndAssertErrorThrown(String statement, String err_msg) throws Exception{ boolean error_thrown = false; try (Statement setupStatement = connection.createStatement()) {