11package io .connect .scylladb ;
22
33import java .io .File ;
4+ import java .util .Collections ;
45import java .util .HashMap ;
56import java .util .List ;
67import java .util .Map ;
@@ -47,18 +48,23 @@ public class ScyllaDbSinkConnectorConfig extends AbstractConfig {
4748 public final TableOptions .CompressionOptions tableCompressionAlgorithm ;
4849 public final char [] trustStorePassword ;
4950 public final File trustStorePath ;
51+ public final char [] keyStorePassword ;
52+ public final File keyStorePath ;
5053 public final String offsetStorageTable ;
5154 public final long statementTimeoutMs ;
5255 public final int maxBatchSizeKb ;
5356 public final String loadBalancingLocalDc ;
5457 public final long timestampResolutionMs ;
5558 public final Map <String , TopicConfigs > topicWiseConfigs ;
5659 public final Integer ttl ;
60+ public final BehaviorOnError behaviourOnError ;
61+ public final List <String > cipherSuites ;
62+ public final File certFilePath ;
63+ public final File privateKeyPath ;
5764
5865 private static final Pattern TOPIC_KS_TABLE_SETTING_PATTERN =
5966 Pattern .compile ("topic\\ .([a-zA-Z0-9._-]+)\\ .([^.]+|\" [\" ]+\" )\\ .([^.]+|\" [\" ]+\" )\\ .(mapping|consistencyLevel|ttlSeconds|deletesEnabled)$" );
6067
61-
6268 static final Map <String , ProtocolOptions .Compression > CLIENT_COMPRESSION =
6369 ImmutableMap .of (
6470 "NONE" , ProtocolOptions .Compression .NONE ,
@@ -95,6 +101,18 @@ public ScyllaDbSinkConnectorConfig(Map<?, ?> originals) {
95101 this .trustStorePassword =
96102 this .getPassword (SSL_TRUSTSTORE_PASSWORD_CONFIG ).value ().toCharArray ();
97103
104+ final String keyStorePath = this .getString (SSL_KEYSTORE_PATH_CONFIG );
105+ this .keyStorePath = Strings .isNullOrEmpty (keyStorePath ) ? null : new File (keyStorePath );
106+ this .keyStorePassword =
107+ this .getPassword (SSL_KEYSTORE_PASSWORD_CONFIG ).value ().toCharArray ();
108+
109+ this .cipherSuites = getList (SSL_CIPHER_SUITES_CONFIG );
110+
111+ final String certFilePath = this .getString (SSL_OPENSLL_KEYCERTCHAIN_CONFIG );
112+ this .certFilePath = Strings .isNullOrEmpty (certFilePath ) ? null : new File (certFilePath );
113+ final String privateKeyPath = this .getString (SSL_OPENSLL_PRIVATEKEY_CONFIG );
114+ this .privateKeyPath = Strings .isNullOrEmpty (privateKeyPath ) ? null : new File (privateKeyPath );
115+
98116 final String compression = getString (COMPRESSION_CONFIG );
99117 this .compression = CLIENT_COMPRESSION .get (compression );
100118 this .sslProvider = ConfigUtils .getEnum (SslProvider .class , this , SSL_PROVIDER_CONFIG );
@@ -127,6 +145,8 @@ public ScyllaDbSinkConnectorConfig(Map<?, ?> originals) {
127145 this .maxBatchSizeKb = getInt (MAX_BATCH_SIZE_CONFIG );
128146 this .loadBalancingLocalDc = getString (LOAD_BALANCING_LOCAL_DC_CONFIG );
129147 this .timestampResolutionMs = getLong (TIMESTAMP_RESOLUTION_MS_CONF );
148+ this .behaviourOnError = BehaviorOnError .valueOf (getString (BEHAVIOR_ON_ERROR_CONFIG ).toUpperCase ());
149+
130150 Map <String , Map <String , String >> topicWiseConfigsMap = new HashMap <>();
131151 for (final Map .Entry <String , String > entry : ((Map <String , String >) originals ).entrySet ()) {
132152 final String name2 = entry .getKey ();
@@ -236,6 +256,22 @@ public ScyllaDbSinkConnectorConfig(Map<?, ?> originals) {
236256 public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "scylladb.ssl.truststore.password" ;
237257 private static final String SSL_TRUSTSTORE_PASSWORD_DOC = "Password to open the Java Truststore with." ;
238258
259+ public static final String SSL_KEYSTORE_PATH_CONFIG = "scylladb.ssl.keystore.path" ;
260+ private static final String SSL_KEYSTORE_PATH_DOC = "Path to the Java Keystore" ;
261+
262+ public static final String SSL_KEYSTORE_PASSWORD_CONFIG = "scylladb.ssl.keystore.password" ;
263+ private static final String SSL_KEYSTORE_PASSWORD_DOC = "Password to open the Java Keystore with." ;
264+
265+ public static final String SSL_CIPHER_SUITES_CONFIG = "scylladb.ssl.cipherSuites" ;
266+ private static final String SSL_CIPHER_SUITES_DOC = "The cipher suites to enable. "
267+ + "Defaults to none, resulting in a ``minimal quality of service`` according to JDK documentation." ;
268+
269+ public static final String SSL_OPENSLL_KEYCERTCHAIN_CONFIG = "scylladb.ssl.openssl.keyCertChain" ;
270+ private static final String SSL_OPENSLL_KEYCERTCHAIN_DOC = "Path to the SSL certificate file, when using OpenSSL." ;
271+
272+ public static final String SSL_OPENSLL_PRIVATEKEY_CONFIG = "ssl.openssl.privateKey" ;
273+ private static final String SSL_OPENSLL_PRIVATEKEY_DOC = "Path to the private key file, when using OpenSSL." ;
274+
239275 public static final String TTL_CONFIG = "scylladb.ttl" ;
240276 /*If TTL value is not specified then skip setting ttl value while making insert query*/
241277 public static final String TTL_DEFAULT = null ;
@@ -263,6 +299,23 @@ public ScyllaDbSinkConnectorConfig(Map<?, ?> originals) {
263299 + "local to the machine on which the connector is running. It is a recommended config if "
264300 + "we have more than one DC." ;
265301
302+ public static final String BEHAVIOR_ON_ERROR_CONFIG = "behavior.on.error" ;
303+ public static final String BEHAVIOR_ON_ERROR_DEFAULT = BehaviorOnError .FAIL .name ();
304+ private static final String BEHAVIOR_ON_ERROR_DISPLAY = "Behavior On Error" ;
305+ private static final String BEHAVIOR_ON_ERROR_DOC = "Error handling behavior setting. "
306+ + "Must be configured to one of the following:\n "
307+ + "``fail``\n "
308+ + "The Connector throws ConnectException and stops processing records "
309+ + "when an error occurs while processing or inserting records into ScyllDB.\n "
310+ + "``ignore``\n "
311+ + "Continues to process next set of records "
312+ + "when error occurs while processing or inserting records into ScyllDB.\n "
313+ + "``log``\n "
314+ + "Logs the error via connect-reporter when an error occurs while processing or "
315+ + "inserting records into ScyllDB and continues to process next set of records, "
316+ + "available in the kafka topics." ;
317+
318+ public static final String SCYLLADB_GROUP = "ScyllaDB" ;
266319 public static final String CONNECTION_GROUP = "Connection" ;
267320 public static final String SSL_GROUP = "SSL" ;
268321 public static final String KEYSPACE_GROUP = "Keyspace" ;
@@ -389,6 +442,56 @@ public static ConfigDef config() {
389442 "SSL Truststore Password" )
390443 //TODO .validator(Validators.blankOr(ValidFile.of()))
391444 //TODO .recommender(Recommenders.visibleIf(SSL_ENABLED_CONFIG, true))
445+ .define (
446+ SSL_KEYSTORE_PATH_CONFIG ,
447+ ConfigDef .Type .STRING ,
448+ "" ,
449+ ConfigDef .Importance .MEDIUM ,
450+ SSL_KEYSTORE_PATH_DOC ,
451+ SSL_GROUP ,
452+ 2 ,
453+ ConfigDef .Width .SHORT ,
454+ "SSL Keystore Path" )
455+ .define (
456+ SSL_KEYSTORE_PASSWORD_CONFIG ,
457+ ConfigDef .Type .PASSWORD ,
458+ "password123" ,
459+ ConfigDef .Importance .MEDIUM ,
460+ SSL_KEYSTORE_PASSWORD_DOC ,
461+ SSL_GROUP ,
462+ 3 ,
463+ ConfigDef .Width .SHORT ,
464+ "SSL Keystore Password" )
465+ .define (
466+ SSL_CIPHER_SUITES_CONFIG ,
467+ ConfigDef .Type .LIST ,
468+ (Object ) Collections .EMPTY_LIST ,
469+ ConfigDef .Importance .HIGH ,
470+ SSL_CIPHER_SUITES_DOC ,
471+ SSL_GROUP ,
472+ 4 ,
473+ ConfigDef .Width .LONG ,
474+ "The cipher suites to enable" )
475+ .define (
476+ SSL_OPENSLL_KEYCERTCHAIN_CONFIG ,
477+ ConfigDef .Type .STRING ,
478+ "" ,
479+ ConfigDef .Importance .HIGH ,
480+ SSL_OPENSLL_KEYCERTCHAIN_DOC ,
481+ SSL_GROUP ,
482+ 5 ,
483+ ConfigDef .Width .SHORT ,
484+ "The path to the certificate chain file" )
485+ .define (
486+ SSL_OPENSLL_PRIVATEKEY_CONFIG ,
487+ ConfigDef .Type .STRING ,
488+ "" ,
489+ ConfigDef .Importance .HIGH ,
490+ SSL_OPENSLL_PRIVATEKEY_DOC ,
491+ SSL_GROUP ,
492+ 6 ,
493+ ConfigDef .Width .SHORT ,
494+ "The path to the private key file" )
392495 .define (
393496 CONSISTENCY_LEVEL_CONFIG ,
394497 ConfigDef .Type .STRING ,
@@ -524,7 +627,21 @@ public static ConfigDef config() {
524627 WRITE_GROUP ,
525628 6 ,
526629 ConfigDef .Width .SHORT ,
527- "Timestamp Threshold in MS" );
630+ "Timestamp Threshold in MS" )
631+ .define (
632+ BEHAVIOR_ON_ERROR_CONFIG ,
633+ ConfigDef .Type .STRING ,
634+ BEHAVIOR_ON_ERROR_DEFAULT ,
635+ ConfigDef .ValidString .in (BehaviorOnError .FAIL .name (),
636+ BehaviorOnError .LOG .name (), BehaviorOnError .IGNORE .name ()),
637+ ConfigDef .Importance .MEDIUM ,
638+ BEHAVIOR_ON_ERROR_DOC ,
639+ SCYLLADB_GROUP ,
640+ 0 ,
641+ ConfigDef .Width .NONE ,
642+ BEHAVIOR_ON_ERROR_DISPLAY
643+ //Recommenders.enumValues(BehaviorOnError.class)
644+ );
528645 }
529646
530647 private String tryMatchTopicName (final String name ) {
@@ -535,6 +652,15 @@ private String tryMatchTopicName(final String name) {
535652 throw new IllegalArgumentException ("The setting: " + name + " does not match topic.keyspace.table nor topic.codec regular expression pattern" );
536653 }
537654
655+ /**
656+ * Enums for behavior on error.
657+ */
658+ public enum BehaviorOnError {
659+ IGNORE ,
660+ LOG ,
661+ FAIL
662+ }
663+
538664 public boolean isOffsetEnabledInScyllaDb () {
539665 return getBoolean (ENABLE_OFFSET_STORAGE_TABLE );
540666 }
0 commit comments