Skip to content
This repository has been archived by the owner on Nov 11, 2022. It is now read-only.

Forward-integrate: merge master into v2 #567

Merged
merged 30 commits into from
May 5, 2017
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
03bf9b7
Increase KafkaIO version to 0.2.0
Dec 14, 2016
8fd6661
Update version range dependency to exclude 2.0.0-betaX versions (#528)
davorbonaci Jan 9, 2017
faa4c2e
README.md for contrib/kafka
Jan 9, 2017
2829f00
README.md for contrib/kafka
Jan 9, 2017
3e61162
Fix formatting
Jan 9, 2017
40d174f
review comments
Jan 9, 2017
2e57ab1
Merge pull request #504 from rangadi/kafka_release_notes
dhalperi Jan 18, 2017
efd33cc
Fix HadoopFileSource’s split size estimate (#534)
igorbernstein2 Jan 26, 2017
dbe4644
BigQuery: fix an issue with option propagation and refactor to future…
dhalperi Jan 30, 2017
b4e391e
[BEAM-359] Treat erased type variables as non-deterministic in AvroCo…
ehlyzov Jan 30, 2017
502f99f
fixups
dhalperi Jan 30, 2017
2ea5a23
Fix InProcessPipelineRunner to handle a null subscription
sammcveety Feb 2, 2017
5625ffb
fixups
dhalperi Feb 6, 2017
9c59d78
fixups
dhalperi Feb 6, 2017
20862aa
Fixups
sammcveety Feb 11, 2017
d7a70fe
Fixups
sammcveety Feb 15, 2017
9b9ee0b
Merge pull request #547 from sammcveety/fix_pubsub_inprocess_null_topic
tgroh Feb 15, 2017
fc5fee2
Merge pull request #540 from dhalperi/bigquery-direct-standard-sql
dhalperi Feb 16, 2017
4a9f164
Small fix for BigtableIO.WriteOperation.finalize
gsgalloway Mar 1, 2017
c4bff0b
Merge pull request #555 from gsgalloway/patch-1
dhalperi Mar 2, 2017
4ede280
edited doc
anilmuppalla Apr 4, 2017
15cb364
Merge pull request #562 from anilmuppalla/master
dhalperi Apr 5, 2017
1cb04a6
DataflowPipelineJob: gracefully handle cancellatoin concurrent with t…
dhalperi Apr 7, 2017
67bfc90
Merge pull request #563 from dhalperi/cancel-double
dhalperi Apr 7, 2017
7cecf6e
Cache result of BigQuerySourceBase.split
jkff Apr 19, 2017
2a1627b
Merge pull request #564 from jkff/backport-2594
dhalperi Apr 19, 2017
52e593a
Makes cachedSplitResult transient in BigQuerySourceBase
jkff Apr 21, 2017
fed09e1
Merge pull request #565 from jkff/backport-2635
dhalperi Apr 21, 2017
f89d619
Fix a typo in Count.java
dhalperi Apr 28, 2017
56f542c
Forward-integrate: master to v2
davorbonaci May 5, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.options.ValueProvider;
import com.google.cloud.dataflow.sdk.options.ValueProvider.StaticValueProvider;
import com.google.cloud.dataflow.sdk.transforms.Aggregator;
import com.google.cloud.dataflow.sdk.transforms.Combine;
import com.google.cloud.dataflow.sdk.transforms.DoFn;
Expand Down Expand Up @@ -1290,6 +1291,7 @@ public String getIdLabel() {

@Override
public PCollection<T> apply(PBegin input) {
ValueProvider<SubscriptionPath> subscriptionPath = subscription;
if (subscription == null) {
try {
try (PubsubClient pubsubClient =
Expand All @@ -1299,9 +1301,8 @@ public PCollection<T> apply(PBegin input) {
.as(DataflowPipelineOptions.class))) {
checkState(project.isAccessible(), "createRandomSubscription must be called at runtime.");
checkState(topic.isAccessible(), "createRandomSubscription must be called at runtime.");
SubscriptionPath subscriptionPath =
pubsubClient.createRandomSubscription(
project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC);
subscriptionPath = StaticValueProvider.of(pubsubClient.createRandomSubscription(
project.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC));
LOG.warn("Created subscription {} to topic {}."
+ " Note this subscription WILL NOT be deleted when the pipeline terminates",
subscription, topic);
Expand All @@ -1314,7 +1315,7 @@ public PCollection<T> apply(PBegin input) {
return input.getPipeline().begin()
.apply(Read.from(new PubsubSource<T>(this)))
.apply(ParDo.named("PubsubUnboundedSource.Stats")
.of(new StatsFn<T>(pubsubFactory, subscription,
timestampLabel, idLabel)));
.of(new StatsFn<T>(pubsubFactory, checkNotNull(subscriptionPath),
timestampLabel, idLabel)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,11 @@ private static class State {
*/
@Nullable
Map<String, Long> ackDeadline;

/**
* Whether a subscription has been created.
*/
boolean createdSubscription;
}

private static final State STATE = new State();
Expand All @@ -124,12 +129,40 @@ public static PubsubTestClientFactory createFactoryForPublish(
final TopicPath expectedTopic,
final Iterable<OutgoingMessage> expectedOutgoingMessages,
final Iterable<OutgoingMessage> failingOutgoingMessages) {
return createFactoryForPublishInternal(
expectedTopic, expectedOutgoingMessages, failingOutgoingMessages, false);
}

/**
* Return a factory for testing publishers. Only one factory may be in-flight at a time.
* The factory must be closed when the test is complete, at which point final validation will
* occur. Additionally, verify that createSubscription was called.
*/
public static PubsubTestClientFactory createFactoryForPublishVerifySubscription(
final TopicPath expectedTopic,
final Iterable<OutgoingMessage> expectedOutgoingMessages,
final Iterable<OutgoingMessage> failingOutgoingMessages) {
return createFactoryForPublishInternal(
expectedTopic, expectedOutgoingMessages, failingOutgoingMessages, true);
}

/**
* Return a factory for testing publishers. Only one factory may be in-flight at a time.
* The factory must be closed when the test is complete, at which point final validation will
* occur.
*/
public static PubsubTestClientFactory createFactoryForPublishInternal(
final TopicPath expectedTopic,
final Iterable<OutgoingMessage> expectedOutgoingMessages,
final Iterable<OutgoingMessage> failingOutgoingMessages,
final boolean verifySubscriptionCreated) {
synchronized (STATE) {
checkState(!STATE.isActive, "Test still in flight");
STATE.expectedTopic = expectedTopic;
STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages);
STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages);
STATE.isActive = true;
STATE.createdSubscription = false;
}
return new PubsubTestClientFactory() {
@Override
Expand All @@ -148,6 +181,9 @@ public String getKind() {
@Override
public void close() {
synchronized (STATE) {
if (verifySubscriptionCreated) {
checkState(STATE.createdSubscription, "Did not call create subscription");
}
checkState(STATE.isActive, "No test still in flight");
checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(),
"Still waiting for %s messages to be published",
Expand Down Expand Up @@ -372,7 +408,10 @@ public List<TopicPath> listTopics(ProjectPath project) throws IOException {
@Override
public void createSubscription(
TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException {
throw new UnsupportedOperationException();
synchronized (STATE) {
STATE.createdSubscription = true;
}
return;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,10 @@
import com.google.cloud.dataflow.sdk.util.CoderUtils;
import com.google.cloud.dataflow.sdk.util.PubsubClient;
import com.google.cloud.dataflow.sdk.util.PubsubClient.IncomingMessage;
import com.google.cloud.dataflow.sdk.util.PubsubClient.OutgoingMessage;
import com.google.cloud.dataflow.sdk.util.PubsubClient.ProjectPath;
import com.google.cloud.dataflow.sdk.util.PubsubClient.SubscriptionPath;
import com.google.cloud.dataflow.sdk.util.PubsubClient.TopicPath;
import com.google.cloud.dataflow.sdk.util.PubsubTestClient;
import com.google.cloud.dataflow.sdk.util.PubsubTestClient.PubsubTestClientFactory;

Expand All @@ -60,8 +63,12 @@
*/
@RunWith(JUnit4.class)
public class PubsubUnboundedSourceTest {
private static final ProjectPath PROJECT =
PubsubClient.projectPathFromId("testProject");
private static final SubscriptionPath SUBSCRIPTION =
PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
private static final TopicPath TOPIC =
PubsubClient.topicPathFromName("testProject", "testTopic");
private static final String DATA = "testData";
private static final long TIMESTAMP = 1234L;
private static final long REQ_TIME = 6373L;
Expand Down Expand Up @@ -320,4 +327,14 @@ public void readManyMessages() throws IOException {
assertTrue(dataToMessageNum.isEmpty());
reader.close();
}

@Test
public void testNullSubscription() throws Exception {
factory = PubsubTestClient.createFactoryForPublishVerifySubscription(
TOPIC, ImmutableList.<OutgoingMessage>of(), ImmutableList.<OutgoingMessage>of());
TestPipeline p = TestPipeline.create();
p.apply(new PubsubUnboundedSource<>(
clock, factory, StaticValueProvider.of(PROJECT), StaticValueProvider.of(TOPIC),
null, StringUtf8Coder.of(), TIMESTAMP_LABEL, ID_LABEL));
}
}