2626import java .util .Map ;
2727import java .util .concurrent .TimeUnit ;
2828import java .util .stream .Stream ;
29+ import org .apache .beam .sdk .io .gcp .spanner .SpannerTransformRegistrar .ChangeStreamReaderBuilder ;
2930import org .apache .beam .sdk .io .gcp .spanner .SpannerTransformRegistrar .InsertBuilder ;
3031import org .apache .beam .sdk .io .gcp .spanner .SpannerTransformRegistrar .ReadBuilder ;
3132import org .apache .beam .sdk .schemas .Schema ;
@@ -48,22 +49,29 @@ public class SpannerTransformRegistrarTest {
4849 public static final String SPANNER_PROJECT = "spanner-project" ;
4950 public static final String SPANNER_TABLE = "spanner-table" ;
5051 public static final String SPANNER_SQL_QUERY = "SELECT * from spanner_table;" ;
52+ public static final String SPANNER_CHANGE_STREAM_NAME = "spanner-change-stream-name" ;
53+ public static final String SPANNER_CHANGE_STREAM_METADATA_INSTANCE =
54+ "spanner-change-stream-instance" ;
55+ public static final String SPANNER_CHANGE_STREAM_METADATA_DATABASE =
56+ "spanner-change-stream-database" ;
5157 private SpannerTransformRegistrar spannerTransformRegistrar ;
5258 private ReadBuilder readBuilder ;
5359 private InsertBuilder writeBuilder ;
60+ private ChangeStreamReaderBuilder changeStreamReaderBuilder ;
5461
5562 @ Before
5663 public void setup () {
5764 spannerTransformRegistrar = new SpannerTransformRegistrar ();
5865 readBuilder = new ReadBuilder ();
5966 writeBuilder = new InsertBuilder ();
67+ changeStreamReaderBuilder = new ChangeStreamReaderBuilder ();
6068 }
6169
6270 @ Test
6371 public void testKnownBuilderInstances () {
6472 Map <String , ExternalTransformBuilder <?, ?, ?>> builderInstancesMap =
6573 spannerTransformRegistrar .knownBuilderInstances ();
66- assertEquals (6 , builderInstancesMap .size ());
74+ assertEquals (7 , builderInstancesMap .size ());
6775 assertThat (builderInstancesMap , IsMapContaining .hasKey (SpannerTransformRegistrar .INSERT_URN ));
6876 assertThat (builderInstancesMap , IsMapContaining .hasKey (SpannerTransformRegistrar .UPDATE_URN ));
6977 assertThat (builderInstancesMap , IsMapContaining .hasKey (SpannerTransformRegistrar .REPLACE_URN ));
@@ -72,6 +80,9 @@ public void testKnownBuilderInstances() {
7280 IsMapContaining .hasKey (SpannerTransformRegistrar .INSERT_OR_UPDATE_URN ));
7381 assertThat (builderInstancesMap , IsMapContaining .hasKey (SpannerTransformRegistrar .DELETE_URN ));
7482 assertThat (builderInstancesMap , IsMapContaining .hasKey (SpannerTransformRegistrar .READ_URN ));
83+ assertThat (
84+ builderInstancesMap ,
85+ IsMapContaining .hasKey (SpannerTransformRegistrar .READ_CHANGE_STREAM_URN ));
7586 }
7687
7788 @ Test (expected = IllegalArgumentException .class )
@@ -207,4 +218,136 @@ private InsertBuilder.Configuration getBasicWriteConfiguration() {
207218 configuration .setMaxCumulativeBackoff (100L );
208219 return configuration ;
209220 }
221+
222+ @ Test (expected = IllegalArgumentException .class )
223+ public void testChangeStreamReaderBuilderBuildExternalWithMissingMandatoryFields () {
224+ changeStreamReaderBuilder .buildExternal (new ChangeStreamReaderBuilder .Configuration ());
225+ }
226+
227+ @ Test (expected = IllegalArgumentException .class )
228+ public void testChangeStreamReaderBuilderBuildExternalWithMissingDatabaseId () {
229+ ChangeStreamReaderBuilder .Configuration configuration =
230+ new ChangeStreamReaderBuilder .Configuration ();
231+ configuration .setProjectId (SPANNER_PROJECT );
232+ configuration .setInstanceId (SPANNER_INSTANCE );
233+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
234+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
235+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
236+ changeStreamReaderBuilder .buildExternal (configuration );
237+ }
238+
239+ @ Test (expected = IllegalArgumentException .class )
240+ public void testChangeStreamReaderBuilderBuildExternalWithMissingInstanceId () {
241+ ChangeStreamReaderBuilder .Configuration configuration =
242+ new ChangeStreamReaderBuilder .Configuration ();
243+ configuration .setProjectId (SPANNER_PROJECT );
244+ configuration .setDatabaseId (SPANNER_DATABASE );
245+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
246+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
247+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
248+ changeStreamReaderBuilder .buildExternal (configuration );
249+ }
250+
251+ @ Test (expected = IllegalArgumentException .class )
252+ public void testChangeStreamReaderBuilderBuildExternalWithMissingChangeStreamName () {
253+ ChangeStreamReaderBuilder .Configuration configuration =
254+ new ChangeStreamReaderBuilder .Configuration ();
255+ configuration .setProjectId (SPANNER_PROJECT );
256+ configuration .setDatabaseId (SPANNER_DATABASE );
257+ configuration .setInstanceId (SPANNER_INSTANCE );
258+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
259+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
260+ changeStreamReaderBuilder .buildExternal (configuration );
261+ }
262+
263+ @ Test (expected = IllegalArgumentException .class )
264+ public void testChangeStreamReaderBuilderBuildExternalWithMissingMetadataInstance () {
265+ ChangeStreamReaderBuilder .Configuration configuration =
266+ new ChangeStreamReaderBuilder .Configuration ();
267+ configuration .setProjectId (SPANNER_PROJECT );
268+ configuration .setDatabaseId (SPANNER_DATABASE );
269+ configuration .setInstanceId (SPANNER_INSTANCE );
270+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
271+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
272+ changeStreamReaderBuilder .buildExternal (configuration );
273+ }
274+
275+ @ Test (expected = IllegalArgumentException .class )
276+ public void testChangeStreamReaderBuilderBuildExternalWithMissingMetadataDatabase () {
277+ ChangeStreamReaderBuilder .Configuration configuration =
278+ new ChangeStreamReaderBuilder .Configuration ();
279+ configuration .setProjectId (SPANNER_PROJECT );
280+ configuration .setDatabaseId (SPANNER_DATABASE );
281+ configuration .setInstanceId (SPANNER_INSTANCE );
282+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
283+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
284+ changeStreamReaderBuilder .buildExternal (configuration );
285+ }
286+
287+ @ Test
288+ public void testChangeStreamReaderBuilderBuildExternalWithRequiredFields () {
289+ ChangeStreamReaderBuilder .Configuration configuration =
290+ new ChangeStreamReaderBuilder .Configuration ();
291+
292+ configuration .setProjectId (SPANNER_PROJECT );
293+ configuration .setDatabaseId (SPANNER_DATABASE );
294+ configuration .setInstanceId (SPANNER_INSTANCE );
295+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
296+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
297+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
298+
299+ PTransform <PBegin , PCollection <String >> changeStreamReaderTransform =
300+ changeStreamReaderBuilder .buildExternal (configuration );
301+ assertNotNull (changeStreamReaderTransform );
302+ }
303+
304+ @ Test
305+ public void testChangeStreamReaderBuilderBuildExternalWithAllFields () {
306+ String startAt = "2023-01-01T00:00:00Z" ;
307+ String endAt = "2023-01-02T00:00:00Z" ;
308+ String metadataTable = "meta-table" ;
309+ String rpcPriority = "HIGH" ;
310+ String refreshRate = "PT30S" ;
311+
312+ ChangeStreamReaderBuilder .Configuration configuration =
313+ new ChangeStreamReaderBuilder .Configuration ();
314+
315+ configuration .setProjectId (SPANNER_PROJECT );
316+ configuration .setDatabaseId (SPANNER_DATABASE );
317+ configuration .setInstanceId (SPANNER_INSTANCE );
318+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
319+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
320+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
321+ configuration .setInclusiveStartAt (startAt );
322+ configuration .setInclusiveEndAt (endAt );
323+ configuration .setMetadataTable (metadataTable );
324+ configuration .setRpcPriority (rpcPriority );
325+ configuration .setWatermarkRefreshRate (refreshRate );
326+
327+ PTransform <PBegin , PCollection <String >> changeStreamReaderTransform =
328+ changeStreamReaderBuilder .buildExternal (configuration );
329+ assertNotNull (changeStreamReaderTransform );
330+ }
331+
332+ @ Test
333+ public void testChangeStreamReaderBuilderBuildExternalWithNullOptionalValues () {
334+ ChangeStreamReaderBuilder .Configuration configuration =
335+ new ChangeStreamReaderBuilder .Configuration ();
336+
337+ configuration .setProjectId (SPANNER_PROJECT );
338+ configuration .setDatabaseId (SPANNER_DATABASE );
339+ configuration .setInstanceId (SPANNER_INSTANCE );
340+ configuration .setChangeStreamName (SPANNER_CHANGE_STREAM_NAME );
341+ configuration .setMetadataInstance (SPANNER_CHANGE_STREAM_METADATA_INSTANCE );
342+ configuration .setMetadataDatabase (SPANNER_CHANGE_STREAM_METADATA_DATABASE );
343+ configuration .setInclusiveStartAt (null );
344+ configuration .setInclusiveEndAt (null );
345+ configuration .setMetadataTable (null );
346+ configuration .setRpcPriority (null );
347+ configuration .setWatermarkRefreshRate (null );
348+
349+ PTransform <PBegin , PCollection <String >> changeStreamReaderTransform =
350+ changeStreamReaderBuilder .buildExternal (configuration );
351+ assertNotNull (changeStreamReaderTransform );
352+ }
210353}
0 commit comments