Skip to content

Commit

Permalink
Add validation that query and table cannot be specified at the same t…
Browse files Browse the repository at this point in the history
…ime for SpannerIO.read() (#31570)
  • Loading branch information
Amar3tto authored Jun 12, 2024
1 parent f4caefe commit 8a64641
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -886,6 +886,10 @@ public PCollection<Struct> expand(PBegin input) {

if (getReadOperation().getQuery() != null) {
// TODO: validate query?
if (getReadOperation().getTable() != null) {
throw new IllegalArgumentException(
"Both query and table cannot be specified at the same time for SpannerIO.read().");
}
} else if (getReadOperation().getTable() != null) {
// Assume read
checkNotNull(
Expand All @@ -898,7 +902,7 @@ public PCollection<Struct> expand(PBegin input) {
+ " list of columns to set with withColumns method");
} else {
throw new IllegalArgumentException(
"SpannerIO.read() requires configuring query or read operation.");
"SpannerIO.read() requires query OR table to set with withTable OR withQuery method.");
}

ReadAll readAll =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.mockito.Mockito;
Expand All @@ -86,6 +87,8 @@ public class SpannerIOReadTest implements Serializable {
public final transient TestPipeline pipeline =
TestPipeline.create().enableAbandonedNodeEnforcement(false);

@Rule public transient ExpectedException thrown = ExpectedException.none();

private FakeServiceFactory serviceFactory;
private BatchReadOnlyTransaction mockBatchTx;
private Partition fakePartition;
Expand Down Expand Up @@ -140,6 +143,22 @@ public void runBatchQueryTestWithSpannerConfig() {
.withTimestampBound(TIMESTAMP_BOUND));
}

@Test
public void runWithQueryAndWithTableAtTheSameTimeFails() {
SpannerIO.Read read =
SpannerIO.read()
.withSpannerConfig(spannerConfig)
.withQuery(QUERY_STATEMENT)
.withQueryName(QUERY_NAME)
.withTable(TABLE_ID)
.withColumns("id", "name")
.withTimestampBound(TIMESTAMP_BOUND);
thrown.expect(IllegalArgumentException.class);
thrown.expectMessage(
"Both query and table cannot be specified at the same time for SpannerIO.read().");
runBatchQueryTest(read);
}

@Test
public void runBatchQueryTestWithUnspecifiedProject() {
// Default spannerConfig has project ID specified - use an unspecified project.
Expand Down

0 comments on commit 8a64641

Please sign in to comment.