Skip to content
This repository was archived by the owner on Nov 11, 2022. It is now read-only.
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
@@ -0,0 +1,23 @@
/*
* Copyright (C) 2015 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/

/**
* Defines a {@link com.google.cloud.dataflow.sdk.coders.Coder}
* for Protocol Buffers messages, {@code ProtoCoder}.
*
* @see com.google.cloud.dataflow.sdk.coders.protobuf.ProtoCoder
*/
package com.google.cloud.dataflow.sdk.coders.protobuf;
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@
* <h3>Reading from Cloud Bigtable</h3>
*
* <p>The Bigtable source returns a set of rows from a single table, returning a
* {@code PCollection&lt;Row&gt;}.
* {@code PCollection<Row>}.
*
* <p>To configure a Cloud Bigtable source, you must supply a table id and a {@link BigtableOptions}
* or builder configured with the project and other information necessary to identify the
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* Copyright (C) 2015 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
* the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/

/**
* Defines transforms for reading and writing from Google Cloud Bigtable.
*
* @see com.google.cloud.dataflow.sdk.io.bigtable.BigtableIO
*/
package com.google.cloud.dataflow.sdk.io.bigtable;
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import com.google.cloud.dataflow.sdk.io.Read.Bounded;
import com.google.cloud.dataflow.sdk.io.Source.Reader;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
*/
package com.google.cloud.dataflow.sdk.runners.inprocess;

import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;

import java.util.Objects;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
package com.google.cloud.dataflow.sdk.runners.inprocess;

import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.Flatten;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import com.google.cloud.dataflow.sdk.coders.IterableCoder;
import com.google.cloud.dataflow.sdk.coders.KvCoder;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.InProcessEvaluationContext;
import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.UncommittedBundle;
import com.google.cloud.dataflow.sdk.runners.inprocess.StepTransformResult.Builder;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1209,8 +1209,11 @@ public TimerUpdateBuilder deletedTimer(TimerData deletedTimer) {
* and deletedTimers.
*/
public TimerUpdate build() {
return new TimerUpdate(key, ImmutableSet.copyOf(completedTimers),
ImmutableSet.copyOf(setTimers), ImmutableSet.copyOf(deletedTimers));
return new TimerUpdate(
key,
ImmutableSet.copyOf(completedTimers),
ImmutableSet.copyOf(setTimers),
ImmutableSet.copyOf(deletedTimers));
}
}

Expand Down Expand Up @@ -1245,6 +1248,13 @@ Iterable<? extends TimerData> getDeletedTimers() {
return deletedTimers;
}

/**
* Returns a {@link TimerUpdate} that is like this one, but with the specified completed timers.
*/
public TimerUpdate withCompletedTimers(Iterable<TimerData> completedTimers) {
return new TimerUpdate(this.key, completedTimers, setTimers, deletedTimers);
}

@Override
public int hashCode() {
return Objects.hash(key, completedTimers, setTimers, deletedTimers);
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (C) 2015 Google Inc.
* Copyright (C) 2016 Google Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not
* use this file except in compliance with the License. You may obtain a copy of
Expand All @@ -22,7 +22,6 @@
import com.google.cloud.dataflow.sdk.util.WindowedValue;
import com.google.cloud.dataflow.sdk.values.PCollection;
import com.google.common.base.MoreObjects;
import com.google.common.base.MoreObjects.ToStringHelper;
import com.google.common.collect.ImmutableList;

import org.joda.time.Instant;
Expand Down Expand Up @@ -64,6 +63,11 @@ private InProcessBundle(PCollection<T> pcollection, boolean keyed, Object key) {
this.elements = ImmutableList.builder();
}

@Override
public PCollection<T> getPCollection() {
return pcollection;
}

@Override
public InProcessBundle<T> add(WindowedValue<T> element) {
checkState(!committed, "Can't add element %s to committed bundle %s", element, this);
Expand Down Expand Up @@ -105,12 +109,12 @@ public Instant getSynchronizedProcessingOutputWatermark() {

@Override
public String toString() {
ToStringHelper toStringHelper =
MoreObjects.toStringHelper(this).add("pcollection", pcollection);
if (keyed) {
toStringHelper = toStringHelper.add("key", key);
}
return toStringHelper.add("elements", elements).toString();
return MoreObjects.toStringHelper(this)
.omitNullValues()
.add("pcollection", pcollection)
.add("key", key)
.add("elements", committedElements)
.toString();
}
};
}
Expand Down
Loading