1
1
package org .apache .kafka .connect .mongodb ;
2
2
3
+ import org .apache .kafka .common .config .ConfigDef ;
3
4
import org .apache .kafka .common .utils .AppInfoParser ;
4
5
import org .apache .kafka .connect .connector .Task ;
5
6
import org .apache .kafka .connect .errors .ConnectException ;
7
+ import org .apache .kafka .connect .mongodb .configuration .MongoDBSinkConfiguration ;
6
8
import org .apache .kafka .connect .sink .SinkConnector ;
7
9
import org .apache .kafka .connect .util .ConnectorUtils ;
8
10
import org .apache .kafka .connect .utils .LogUtils ;
9
11
import org .apache .kafka .connect .utils .StringUtils ;
10
12
import org .slf4j .Logger ;
11
13
import org .slf4j .LoggerFactory ;
12
14
13
- import java .util .*;
15
+ import java .util .ArrayList ;
16
+ import java .util .Arrays ;
17
+ import java .util .HashMap ;
18
+ import java .util .List ;
19
+ import java .util .Map ;
20
+ import java .util .stream .IntStream ;
14
21
15
22
/**
16
- * MongodbSinkConnector implement the Connector interface to send Kafka
23
+ * MongoDBSinkConnector implement the Connector interface to send Kafka
17
24
* data to Mongodb
18
25
*
19
26
* @author Andrea Patelli
27
+ * @author Niraj Patel
20
28
*/
21
- public class MongodbSinkConnector extends SinkConnector {
22
- private final static Logger log = LoggerFactory .getLogger (MongodbSinkConnector .class );
29
+ public class MongoDBSinkConnector extends SinkConnector {
23
30
24
- public static final String PORT = "port" ;
25
- public static final String HOST = "host" ;
26
- public static final String BULK_SIZE = "bulk.size" ;
27
- public static final String DATABASE = "mongodb.database" ;
28
- public static final String COLLECTIONS = "mongodb.collections" ;
29
- public static final String TOPICS = "topics" ;
31
+ private final static Logger log = LoggerFactory .getLogger (MongoDBSinkConnector .class );
30
32
31
- private String port ;
32
- private String host ;
33
- private String bulkSize ;
34
- private String database ;
35
- private String collections ;
36
- private String topics ;
33
+ private Map <String , String > configuration ;
37
34
38
35
/**
39
36
* Get the version of this connector.
@@ -49,33 +46,20 @@ public String version() {
49
46
* Start this Connector. This method will only be called on a clean Connector, i.e. it has
50
47
* either just been instantiated and initialized or {@link #stop()} has been invoked.
51
48
*
52
- * @param map configuration settings
49
+ * @param configuration configuration settings
53
50
*/
54
51
@ Override
55
- public void start (Map <String , String > map ) {
56
- log .trace ("Parsing configuration" );
57
-
58
- port = map .get (PORT );
59
- if (port == null || port .isEmpty ())
60
- throw new ConnectException ("Missing " + PORT + " config" );
61
-
62
- bulkSize = map .get (BULK_SIZE );
63
- if (bulkSize == null || bulkSize .isEmpty ())
64
- throw new ConnectException ("Missing " + BULK_SIZE + " config" );
65
-
66
- host = map .get (HOST );
67
- if (host == null || host .isEmpty ())
68
- throw new ConnectException ("Missing " + HOST + " config" );
69
-
70
- database = map .get (DATABASE );
71
- collections = map .get (COLLECTIONS );
72
- topics = map .get (TOPICS );
52
+ public void start (Map <String , String > configuration ) {
53
+ MongoDBSinkConfiguration sinkConfiguration = new MongoDBSinkConfiguration (configuration );
54
+ this .configuration = sinkConfiguration .originalsStrings ();
73
55
56
+ String collections = configuration .get (MongoDBSinkConfiguration .COLLECTIONS_CONFIG );
57
+ String topics = configuration .get (MongoDBSinkConfiguration .TOPICS_CONFIG );
74
58
if (collections .split ("," ).length != topics .split ("," ).length ) {
75
59
throw new ConnectException ("The number of topics should be the same as the number of collections" );
76
60
}
77
61
78
- LogUtils .dumpConfiguration (map , log );
62
+ LogUtils .dumpConfiguration (configuration , log );
79
63
}
80
64
81
65
/**
@@ -85,7 +69,7 @@ public void start(Map<String, String> map) {
85
69
*/
86
70
@ Override
87
71
public Class <? extends Task > taskClass () {
88
- return MongodbSinkTask .class ;
72
+ return MongoDBSinkTask .class ;
89
73
}
90
74
91
75
/**
@@ -98,22 +82,21 @@ public Class<? extends Task> taskClass() {
98
82
@ Override
99
83
public List <Map <String , String >> taskConfigs (int maxTasks ) {
100
84
List <Map <String , String >> configs = new ArrayList <>();
101
- List <String > coll = Arrays .asList (collections .split ("," ));
85
+ List <String > coll = Arrays .asList (configuration . get ( MongoDBSinkConfiguration . COLLECTIONS_CONFIG ) .split ("," ));
102
86
int numGroups = Math .min (coll .size (), maxTasks );
103
87
List <List <String >> dbsGrouped = ConnectorUtils .groupPartitions (coll , numGroups );
104
- List <String > topics = Arrays .asList (this . topics .split ("," ));
88
+ List <String > topics = Arrays .asList (configuration . get ( MongoDBSinkConfiguration . TOPICS_CONFIG ) .split ("," ));
105
89
List <List <String >> topicsGrouped = ConnectorUtils .groupPartitions (topics , numGroups );
106
90
107
- for ( int i = 0 ; i < numGroups ; i ++) {
91
+ IntStream . range ( 0 , numGroups ). forEach ( i -> {
108
92
Map <String , String > config = new HashMap <>();
109
- config .put (PORT , port );
110
- config .put (BULK_SIZE , bulkSize );
111
- config .put (HOST , host );
112
- config .put (DATABASE , database );
113
- config .put (COLLECTIONS , StringUtils .join (dbsGrouped .get (i ), "," ));
114
- config .put (TOPICS , StringUtils .join (topicsGrouped .get (i ), "," ));
93
+ config .put (MongoDBSinkConfiguration .BULK_SIZE_CONFIG , configuration .get (MongoDBSinkConfiguration .BULK_SIZE_CONFIG ));
94
+ config .put (MongoDBSinkConfiguration .HOST_URLS_CONFIG , configuration .get (MongoDBSinkConfiguration .HOST_URLS_CONFIG ));
95
+ config .put (MongoDBSinkConfiguration .DATABASE_CONFIG , configuration .get (MongoDBSinkConfiguration .DATABASE_CONFIG ));
96
+ config .put (MongoDBSinkConfiguration .COLLECTIONS_CONFIG , StringUtils .join (dbsGrouped .get (i ), "," ));
97
+ config .put (MongoDBSinkConfiguration .TOPICS_CONFIG , StringUtils .join (topicsGrouped .get (i ), "," ));
115
98
configs .add (config );
116
- }
99
+ });
117
100
return configs ;
118
101
}
119
102
@@ -122,6 +105,12 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
122
105
*/
123
106
@ Override
124
107
public void stop () {
108
+ // not implemented
109
+ }
125
110
111
+ @ Override
112
+ public ConfigDef config () {
113
+ return MongoDBSinkConfiguration .config ;
126
114
}
115
+
127
116
}
0 commit comments