Skip to content

Commit

Permalink
Addressed review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
peihe committed Apr 8, 2016
1 parent ff7706e commit 24fc00e
Show file tree
Hide file tree
Showing 3 changed files with 6 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -788,14 +788,14 @@ public Bound() {
private Bound(String name, @Nullable String jsonTableRef,
@Nullable SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
@Nullable String jsonSchema,
CreateDisposition createDisposition, WriteDisposition writeDisposition,
boolean validate, BigQueryServices testBigQueryServices) {
CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate,
@Nullable BigQueryServices testBigQueryServices) {
super(name);
this.jsonTableRef = jsonTableRef;
this.tableRefFunction = tableRefFunction;
this.jsonSchema = jsonSchema;
this.createDisposition = createDisposition;
this.writeDisposition = writeDisposition;
this.createDisposition = checkNotNull(createDisposition, "createDisposition");
this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
this.validate = validate;
this.testBigQueryServices = testBigQueryServices;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import com.google.cloud.dataflow.sdk.options.BigQueryOptions;
import com.google.cloud.hadoop.util.ApiErrorExtractor;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,8 @@
import com.google.cloud.dataflow.sdk.transforms.Create;
import com.google.cloud.dataflow.sdk.util.BigQueryServices;
import com.google.cloud.dataflow.sdk.util.BigQueryServices.Status;
import com.google.common.collect.ImmutableList;
import com.google.cloud.dataflow.sdk.util.CoderUtils;
import com.google.common.collect.ImmutableList;

import org.hamcrest.Matchers;
import org.junit.Assert;
Expand Down Expand Up @@ -196,7 +196,7 @@ private void checkWriteObjectWithValidate(
assertEquals(project, bound.getTable().getProjectId());
assertEquals(dataset, bound.getTable().getDatasetId());
assertEquals(table, bound.getTable().getTableId());
assertEquals(schema, bound.jsonSchema);
assertEquals(schema, bound.getSchema());
assertEquals(createDisposition, bound.createDisposition);
assertEquals(writeDisposition, bound.writeDisposition);
assertEquals(validate, bound.validate);
Expand Down

0 comments on commit 24fc00e

Please sign in to comment.