|
34 | 34 | import com.google.cloud.spanner.Partition; |
35 | 35 | import com.google.cloud.spanner.PartitionOptions; |
36 | 36 | import com.google.cloud.spanner.ResultSets; |
| 37 | +import com.google.cloud.spanner.SpannerException; |
37 | 38 | import com.google.cloud.spanner.SpannerExceptionFactory; |
38 | 39 | import com.google.cloud.spanner.Statement; |
39 | 40 | import com.google.cloud.spanner.Struct; |
40 | 41 | import com.google.cloud.spanner.TimestampBound; |
41 | 42 | import com.google.cloud.spanner.Type; |
42 | 43 | import com.google.cloud.spanner.Value; |
43 | 44 | import com.google.protobuf.ByteString; |
| 45 | +import io.grpc.Status.Code; |
44 | 46 | import java.io.Serializable; |
45 | 47 | import java.util.Arrays; |
46 | 48 | import java.util.HashMap; |
|
49 | 51 | import org.apache.beam.runners.core.metrics.MetricsContainerImpl; |
50 | 52 | import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; |
51 | 53 | import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; |
| 54 | +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; |
52 | 55 | import org.apache.beam.sdk.io.gcp.spanner.SpannerIO.Read; |
53 | 56 | import org.apache.beam.sdk.metrics.MetricsEnvironment; |
54 | 57 | import org.apache.beam.sdk.testing.PAssert; |
@@ -293,7 +296,7 @@ public void runReadWithPriority() throws Exception { |
293 | 296 | } |
294 | 297 |
|
295 | 298 | @Test |
296 | | - public void testQueryMetrics() throws Exception { |
| 299 | + public void testQueryMetricsFail() throws Exception { |
297 | 300 | Timestamp timestamp = Timestamp.ofTimeMicroseconds(12345); |
298 | 301 | TimestampBound timestampBound = TimestampBound.ofReadTimestamp(timestamp); |
299 | 302 |
|
@@ -322,25 +325,74 @@ public void testQueryMetrics() throws Exception { |
322 | 325 | any(PartitionOptions.class), |
323 | 326 | eq(Statement.of("SELECT * FROM users")), |
324 | 327 | any(ReadQueryUpdateTransactionOption.class))) |
325 | | - .thenReturn(Arrays.asList(fakePartition, fakePartition)); |
| 328 | + .thenReturn(Arrays.asList(fakePartition)); |
326 | 329 | when(mockBatchTx.execute(any(Partition.class))) |
327 | 330 | .thenThrow( |
328 | 331 | SpannerExceptionFactory.newSpannerException( |
329 | | - ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 1")) |
330 | | - .thenThrow( |
331 | | - SpannerExceptionFactory.newSpannerException( |
332 | | - ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 2")) |
| 332 | + ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 1")); |
| 333 | + try { |
| 334 | + pipeline.run(); |
| 335 | + } catch (PipelineExecutionException e) { |
| 336 | + if (e.getCause() instanceof SpannerException |
| 337 | + && ((SpannerException) e.getCause()).getErrorCode().getGrpcStatusCode() |
| 338 | + == Code.DEADLINE_EXCEEDED) { |
| 339 | + // expected |
| 340 | + } else { |
| 341 | + throw e; |
| 342 | + } |
| 343 | + } |
| 344 | + verifyMetricWasSet("test", "aaa", "123", "deadline_exceeded", null, 1); |
| 345 | + verifyMetricWasSet("test", "aaa", "123", "ok", null, 0); |
| 346 | + } |
| 347 | + |
| 348 | + @Test |
| 349 | + public void testQueryMetricsSucceed() throws Exception { |
| 350 | + Timestamp timestamp = Timestamp.ofTimeMicroseconds(12345); |
| 351 | + TimestampBound timestampBound = TimestampBound.ofReadTimestamp(timestamp); |
| 352 | + |
| 353 | + SpannerConfig spannerConfig = getSpannerConfig(); |
| 354 | + |
| 355 | + pipeline.apply( |
| 356 | + "read q", |
| 357 | + SpannerIO.read() |
| 358 | + .withSpannerConfig(spannerConfig) |
| 359 | + .withQuery("SELECT * FROM users") |
| 360 | + .withQueryName("queryName") |
| 361 | + .withTimestampBound(timestampBound)); |
| 362 | + |
| 363 | + FakeBatchTransactionId id = new FakeBatchTransactionId("runQueryTest"); |
| 364 | + when(mockBatchTx.getBatchTransactionId()).thenReturn(id); |
| 365 | + |
| 366 | + when(serviceFactory.mockBatchClient().batchReadOnlyTransaction(timestampBound)) |
| 367 | + .thenReturn(mockBatchTx); |
| 368 | + when(serviceFactory.mockBatchClient().batchReadOnlyTransaction(any(BatchTransactionId.class))) |
| 369 | + .thenReturn(mockBatchTx); |
| 370 | + |
| 371 | + Partition fakePartition = |
| 372 | + FakePartitionFactory.createFakeQueryPartition(ByteString.copyFromUtf8("one")); |
| 373 | + |
| 374 | + when(mockBatchTx.partitionQuery( |
| 375 | + any(PartitionOptions.class), |
| 376 | + eq(Statement.of("SELECT * FROM users")), |
| 377 | + any(ReadQueryUpdateTransactionOption.class))) |
| 378 | + .thenReturn(Arrays.asList(fakePartition, fakePartition)); |
| 379 | + when(mockBatchTx.execute(any(Partition.class))) |
333 | 380 | .thenReturn( |
334 | 381 | ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(0, 2)), |
335 | | - ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(2, 6))); |
| 382 | + ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(2, 4)), |
| 383 | + ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(4, 6))) |
| 384 | + .thenReturn( |
| 385 | + ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(0, 2)), |
| 386 | + ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(2, 4)), |
| 387 | + ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(4, 6))); |
336 | 388 |
|
337 | 389 | pipeline.run(); |
338 | | - verifyMetricWasSet("test", "aaa", "123", "deadline_exceeded", null, 2); |
| 390 | + verifyMetricWasSet("test", "aaa", "123", "deadline_exceeded", null, 0); |
339 | 391 | verifyMetricWasSet("test", "aaa", "123", "ok", null, 2); |
340 | 392 | } |
341 | 393 |
|
342 | 394 | @Test |
343 | | - public void testReadMetrics() throws Exception { |
| 395 | + public void testReadMetricsFail() throws Exception { |
344 | 396 | Timestamp timestamp = Timestamp.ofTimeMicroseconds(12345); |
345 | 397 | TimestampBound timestampBound = TimestampBound.ofReadTimestamp(timestamp); |
346 | 398 |
|
@@ -371,21 +423,66 @@ public void testReadMetrics() throws Exception { |
371 | 423 | eq(KeySet.all()), |
372 | 424 | eq(Arrays.asList("id", "name")), |
373 | 425 | any(ReadQueryUpdateTransactionOption.class))) |
374 | | - .thenReturn(Arrays.asList(fakePartition, fakePartition, fakePartition)); |
| 426 | + .thenReturn(Arrays.asList(fakePartition)); |
375 | 427 | when(mockBatchTx.execute(any(Partition.class))) |
376 | 428 | .thenThrow( |
377 | 429 | SpannerExceptionFactory.newSpannerException( |
378 | | - ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 1")) |
379 | | - .thenThrow( |
380 | | - SpannerExceptionFactory.newSpannerException( |
381 | | - ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 2")) |
| 430 | + ErrorCode.DEADLINE_EXCEEDED, "Simulated Timeout 1")); |
| 431 | + try { |
| 432 | + pipeline.run(); |
| 433 | + } catch (PipelineExecutionException e) { |
| 434 | + if (e.getCause() instanceof SpannerException |
| 435 | + && ((SpannerException) e.getCause()).getErrorCode().getGrpcStatusCode() |
| 436 | + == Code.DEADLINE_EXCEEDED) { |
| 437 | + // expected |
| 438 | + } else { |
| 439 | + throw e; |
| 440 | + } |
| 441 | + } |
| 442 | + verifyMetricWasSet("test", "aaa", "123", "deadline_exceeded", null, 1); |
| 443 | + verifyMetricWasSet("test", "aaa", "123", "ok", null, 0); |
| 444 | + } |
| 445 | + |
| 446 | + @Test |
| 447 | + public void testReadMetricsSucceed() throws Exception { |
| 448 | + Timestamp timestamp = Timestamp.ofTimeMicroseconds(12345); |
| 449 | + TimestampBound timestampBound = TimestampBound.ofReadTimestamp(timestamp); |
| 450 | + |
| 451 | + SpannerConfig spannerConfig = getSpannerConfig(); |
| 452 | + |
| 453 | + pipeline.apply( |
| 454 | + "read q", |
| 455 | + SpannerIO.read() |
| 456 | + .withSpannerConfig(spannerConfig) |
| 457 | + .withTable("users") |
| 458 | + .withColumns("id", "name") |
| 459 | + .withTimestampBound(timestampBound)); |
| 460 | + |
| 461 | + FakeBatchTransactionId id = new FakeBatchTransactionId("runReadTest"); |
| 462 | + when(mockBatchTx.getBatchTransactionId()).thenReturn(id); |
| 463 | + |
| 464 | + when(serviceFactory.mockBatchClient().batchReadOnlyTransaction(timestampBound)) |
| 465 | + .thenReturn(mockBatchTx); |
| 466 | + when(serviceFactory.mockBatchClient().batchReadOnlyTransaction(any(BatchTransactionId.class))) |
| 467 | + .thenReturn(mockBatchTx); |
| 468 | + |
| 469 | + Partition fakePartition = |
| 470 | + FakePartitionFactory.createFakeReadPartition(ByteString.copyFromUtf8("one")); |
| 471 | + |
| 472 | + when(mockBatchTx.partitionRead( |
| 473 | + any(PartitionOptions.class), |
| 474 | + eq("users"), |
| 475 | + eq(KeySet.all()), |
| 476 | + eq(Arrays.asList("id", "name")), |
| 477 | + any(ReadQueryUpdateTransactionOption.class))) |
| 478 | + .thenReturn(Arrays.asList(fakePartition, fakePartition, fakePartition)); |
| 479 | + when(mockBatchTx.execute(any(Partition.class))) |
382 | 480 | .thenReturn( |
383 | 481 | ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(0, 2)), |
384 | 482 | ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(2, 4)), |
385 | 483 | ResultSets.forRows(FAKE_TYPE, FAKE_ROWS.subList(4, 6))); |
386 | 484 |
|
387 | 485 | pipeline.run(); |
388 | | - verifyMetricWasSet("test", "aaa", "123", "deadline_exceeded", null, 2); |
389 | 486 | verifyMetricWasSet("test", "aaa", "123", "ok", null, 3); |
390 | 487 | } |
391 | 488 |
|
|
0 commit comments