See <https://builds.apache.org/job/beam/3/changes>

Changes:

[davorbonaci] Add support for resolving paths within IOChannelUtils and 
corresponding

[davorbonaci] Stops using system properties for controlling progress reporting

[davorbonaci] Workflow data disk specification improvements. ----Release 
Notes----

[davorbonaci] Use ExpectedException instead of try/catch/fail.

[davorbonaci] Setting the ReadChannel to null in the case when an SSL exception 
is

[davorbonaci] Add input data watermark to GetWork response. ----Release 
Notes---- []

[davorbonaci] For the PipelineOptionsFactory, provided a clearer exception 
message

[davorbonaci] Updated BigQueryReader so that BigQuery values of type TIMESTAMP 
are

[davorbonaci] Changes logging level of Windmill.CommitWorkRequest to trace - the

[davorbonaci] Stop sending progress updates if a workItem fails. ----Release 
Notes----

[davorbonaci] Change how WindowFn's report their properties.

[davorbonaci] Set maxTimestamp of GlobalWindow to be one day before max.

[davorbonaci] Move GroupByKey expansion to the backend. This passes in windowFn

[davorbonaci] Add AccumulationMode enum and support for accumulating.

[davorbonaci] Improve the description of Window.into operations.

[davorbonaci] Use TypeVariable instead of String to name generic parameters 
during

[davorbonaci] Verify the status of windows at various points in Trigger 
Execution.

[davorbonaci] When reading from source with gzip Content-Encoding, always 
request the

[davorbonaci] Fix wrongly set buffer size for GoogleCloudStorageWriteChannel.

[davorbonaci] Add MapFn and FlatMapFn

[davorbonaci] Migrate the package stager to use IOChannelUtils so that it can 
stage

[davorbonaci] Expose Accumulating Mode via the Window.into operation.

[davorbonaci] Refactor the existing WindowSets.

[davorbonaci] Centralize the choice of what window set to use.

[davorbonaci] Fix test failure issues on windows because of paths.

[davorbonaci] Revert "Add MapFn and FlatMapFn"

[davorbonaci] Only invoke waitToFinish() when DataflowPipelineJob is provided.

[davorbonaci] Apply Maven shading to the Guava dependency library.

[davorbonaci] Update test_wordcount.sh following replacement of bundling with 
shading

[alex] Ignore additional IntelliJ files.

[davorbonaci] Add newline at the end of .gitignore

[davorbonaci] Fix Javadoc errors that show up with JDK 8.

[davorbonaci] Document the immutability of input/output elements.

[davorbonaci] Remove NoopStager. Warn if classpath contains non-existent 
resources.

[davorbonaci] Remove unused method that leaked a dependency into the API 
surface.

[davorbonaci] Use application default credentials from gcloud core component.

[davorbonaci] Expose input watermark in StreamingModeExecutionContext

[davorbonaci] Allows a PTransform to be used more than once

[davorbonaci] De-whitelist org.codehaus.jackson in ApiSurface

[davorbonaci] Remove extraneous jackson leakage from CloudDebuggerOptions

[davorbonaci] Limit jackson leakage to annotations

[davorbonaci] Introduce isSplittable for FileBasedSources

[davorbonaci] Add Maven rules for producing Javadoc

[davorbonaci] Print the CLI command for cancelling jobs running in the cloud

[davorbonaci] Rename getWatermarkCutoff to getWatermarkThatGuaranteesFiring

[davorbonaci] Update comment on the shutdown hook

[davorbonaci] Make tests locale-independent

[davorbonaci] Fix dataflow cancel command suggestion message

[davorbonaci] Have WindowingInternals provide the TimerManager

[davorbonaci] Decrease number of threads created in test

[davorbonaci] Move TimeDomain into TimerManager

[davorbonaci] Fix a typo in the java doc. ----Release Notes---- [] -------------

[davorbonaci] Remove unused SuppressWarnings annotations

[davorbonaci] Mark PCollections as bounded or unbounded

[davorbonaci] Fix a typo in the java doc.

[davorbonaci] Fix javadoc errors that surface with JDK8

[davorbonaci] Use logging rather than stdout in StarterPipeline

[davorbonaci] Fix error in SDK javadoc surfaced via JDK 8

[davorbonaci] Fix a typo in the java doc.

[davorbonaci] Add AggregatorPipelineExtractor

[davorbonaci] Ignore empty arguments when parsing PipelineOptions

[davorbonaci] Log the number of matched files for a filepattern

[davorbonaci] Fix a typo in the error message.

[davorbonaci] Add JobSpecification to DataflowPipelineTranslator

[davorbonaci] Always produce at least one split of a Datastore query

[davorbonaci] Fix parameter name in AutoCompleteTest

[davorbonaci] Improve toString on *Pipeline and *Runner classes

[davorbonaci] Expand JavaDoc around PipelineOptions

[davorbonaci] Make non-static TupleTag test robust to superficial changes

[davorbonaci] Make the StateSampler available to the abstract Reader 
super-class so it

[davorbonaci] Updated KMean to do Windowing, and be able to run in streaming.

[davorbonaci] Add Aggregator Value lookups

[davorbonaci] Fix documentation and change visibility of FileResult

[davorbonaci] Remove all uses of deprecated PTransform#withName

[davorbonaci] Add allowedLateness to WindowingStrategy

[davorbonaci] Remove the experimental KeyedState API from DoFns

[davorbonaci] Get security fixes from newer version of Jetty

[davorbonaci] Fix JDK8-specific Javadoc errors

[davorbonaci] Improve Trigger interaction with the Watermark

[davorbonaci] Makes Write compatible with windowing

[davorbonaci] Move experimental late-data handling from PubSubIO to Window.

[davorbonaci] Fix incorrect use of effect/affect

[davorbonaci] Update maven surefire plugin

[davorbonaci] Link from our javadoc to that of Hamcrest and JUnit.

[davorbonaci] Change ExpectedLogs to include better stack-traces

[davorbonaci] Introduces CompressedSource

[davorbonaci] Ensure at least one split is produced in DatastoreIO

[davorbonaci] Add option to treat non-stable-unique name as an error

[davorbonaci] Allow overriding transform name in apply

[davorbonaci] Adds ConcatReader - a Reader that may encapsulate one or more 
Dataflow

[davorbonaci] Use PTransforms inside DataflowAssert to scope operations

[davorbonaci] Update BigQueryTableRowIterator so that it retries BigQuery 
requests

[davorbonaci] Fix flaky PipelineOptionsFactoryTest order issue

[davorbonaci] Include BASIC importance in the messages printed by PrintHandler

[davorbonaci] Clarifies that TextIO.Write will overwrite existing files

[davorbonaci] Add withCoder to Create and CreateTimestamped

[davorbonaci] Windmill API changes for unbounded custom sources

[davorbonaci] Windmill API changes to support client-side caching.

[davorbonaci] Pushes bounded-only Source methods into BoundedSource

[davorbonaci] Fix stable-unique-name errors in unit tests

[davorbonaci] Caches the Content-Encoding of GCS files

[davorbonaci] Remove PTransform#withName and PTransform#setName

[davorbonaci] DataflowWorker.handleWorkError should use nextReportIndex

[davorbonaci] Improve user error message when a GCS path is expected

[davorbonaci] Testing improvements regarding DataflowAssert and TestPipeline

[davorbonaci] Only take ownership of argument to ByteArrayCoder explicitly

[davorbonaci] Fix handling of key token invalid errors

[davorbonaci] Adds support for PubsubIO in batch and direct modes.

[davorbonaci] Cleanup stable name warnings in `mvn clean install`

[davorbonaci] Update ExamplePubsubTopicOptions to use pubsub v1beta2 format

[davorbonaci] Add PCollectionTuple#apply(String, PTransform) and improve javadoc

[davorbonaci] Proto changes for dependent system watermarks and dependent 
realtime

[davorbonaci] Unflake DataflowWorkProgressUpdaterTest and minor cleanup

[davorbonaci] Improve View.asMap docs, interface, and examples

[davorbonaci] BlockingDataflowPipelineRunner throw exception if pipeline fails

[davorbonaci] Introduces additional source testing utilities

[davorbonaci] Support empty composite transforms; tolerate empty pipelines

[davorbonaci] Improve error for failing to serialize PipelineOptions

[davorbonaci] Update generated Dataflow API from v1beta3 to v1b3

[davorbonaci] Revert "Improve error for failing to serialize PipelineOptions"

[davorbonaci] Cancel command is printed with the endpoint.

[davorbonaci] Fix javadoc issue with @

[davorbonaci] Internal test changes

[davorbonaci] Adds PipelineOption to specify a GCE network for GCE VMs

[davorbonaci] Add logic in SDK to support unique name check during job creation.

[davorbonaci] Improve error for failing to serialize PipelineOptions

[davorbonaci] Adds source testing utilities for checking source splitAtFraction.

[davorbonaci] Update version of protobufs used by the SDK

[davorbonaci] Updates XmlSourceTest so that it passes for non-english locales.

[davorbonaci] Fix missing table creations in BigQueryIO.Write.

[davorbonaci] Turning off the 250Gb upload limit by default

[davorbonaci] Make ParDoFnFactory a real class; remove some reflection

[davorbonaci] Fix a typo in Aggregator java doc

[davorbonaci] Change LOG.info to LOG.debug to avoid the log spams

[davorbonaci] Fixes many occurrences of missing try-with-resources

[davorbonaci] Fix broken @link

[davorbonaci] Specify javadoc plugin version for the examples module

[lcwik] Fix the instructions to override the API endpoint

[lcwik] Enable Proto2Coder and SerializerTest

[lcwik] Introduces a custom source implementation of Avro

[lcwik] Support per-window tables in BigQueryIO

[lcwik] Do not use the Datastore query splitter for one split

[lcwik] Improve key throughput of StreamingDataflowWorker

[lcwik] Separate ActiveWindowSet and OutputBuffers

[lcwik] Make inner Coder class in Top transform static

[lcwik] Update option to turn on Cloud Debugger for Dataflow jobs

[lcwik] Move streaming custom transform expansions into the runner

[lcwik] Three successively more detailed WordCount examples

[lcwik] PTransform.getName() refactoring and fixes for readable names in UI

[lcwik] Remove time-dependent parts of ReadOperationTest

[lcwik] Update internal documentation

[lcwik] Internal test changes.

[lcwik] Explicitly require Project in Dataflow Options

[lcwik] Fixes the flaky test XmlSourceTest.testSplitAtFraction.

[lcwik] For batch execution, replace fixed side input Map with a Cache

[lcwik] Drop filesToStage from serialized pipeline options

[lcwik] Improve type safety of coder instance registration

[lcwik] Implement GroupAlsoByWindowsAndCombineDoFn

[lcwik] Renamed --input to --inputFile for WordCount

[lcwik] Expose dataset ID in ApplianceShuffleWriter

[lcwik] Adds support for unbounded custom sources

[lcwik] Make HolderCoder#equals depend on the inputCoder

[lcwik] Make ParDoFn an interface

[lcwik] Clean up StreamingDataflowWorkerTest

[lcwik] Serialize Proto2Coder via JSON, not Java serialization

[lcwik] Remove FixedIntervalWindowCoder

[lcwik] Cookbook examples reorganization

[lcwik] Add plumbing for pane/trigger tags

[lcwik] Convert PipelineOptions to Output Type Pre-validation

[lcwik] Verify that Window Coders are deterministic

[lcwik] Trigger-defined "continuation" behavior

[lcwik] Update option enabling Cloud Debugger for Dataflow jobs

[lcwik] Remove unnecessary dependencies

[lcwik] Internal testing change

[lcwik] Enable group also by window combiner lifting

[lcwik] Update GroupAlsoByWindowsAndCombineDoFn to support merging windows

[lcwik] Add a status server to the batch DataflowWorker

[lcwik] Update GroupAlsoByWindowsAndCombineDoFn to support accumulation mode

[lcwik] Update GABWViaIteratorsDoFn for ACCUMULATING_FIRED_PANES

[lcwik] Serialize JAXBCoder via JSON, not Java serialization

[lcwik] Test @DefaultCoder for collection contents

[lcwik] Make inner Coder class in ApproximateQuantiles static

[lcwik] Add counters for data that is dropped in GroupAlsoByWindows

[lcwik] Add Group as an attribute of Validation.Required

[lcwik] Introduces RangeTracker for thread-safe dynamic splitting

[lcwik] Makes dynamic splits non-blocking

[lcwik] Windmill API changes for supporting explicit record ids.

[lcwik] Adding in a utility class to help with gcs retries

[lcwik] Expose CoderRegistry.getDefaultCoder(Class)

[lcwik] Fix Javadoc in Window

[lcwik] Propagate CoderException from Coder through CoderUtils

[lcwik] Make anonymous inner Coder class in Mean named and static

[lcwik] Remove unneeded warning-causing code in CoderRegistry

[lcwik] Add a transform name mapping for updating streaming pipelines

[lcwik] Rename TriggerExecutor.trigger to TriggerExecutor.rootTrigger

[lcwik] Make TriggerContext an abstract class

[lcwik] Combine the On*Event and TriggerContext interfaces

[lcwik] Allow overwrite of registered Coder

[lcwik] Make the window available in the TriggerContext

[lcwik] Remove window parameter from Timer methods

[lcwik] Cleanup hashCode and equals for some triggers

[lcwik] Move Timer and Merge specific methods into appropriate Contexts

[lcwik] Fix an invalid path on Windows platforms

[lcwik] Move OrFinallyTrigger into its own file

[lcwik] Clean up resources immediately and some small code cleanup

[lcwik] Windmill.proto changes. Adding counter field in ReportStatsRequest.

[lcwik] Windmill API changes for supporting timestamps and ids in

[lcwik] Implementation of shuffle load balancing for Dataflow

[lcwik] Windmill API changes for using system names for state.

[lcwik] Update Windmill API protos.

[lcwik] Use system names instead of user names for storing state

[lcwik] Adds SDK support for creating BigQuery sources using queries

[lcwik] Log structured metadata for Cloud Logging

[lcwik] Bound the exponential backoff time for work requests

[lcwik] Internal testing changes

[lcwik] Plumb timestampLabel and idLabel through PubsubSink.

[lcwik] Fix incorrect "cast" for StreamingOptions

[lcwik] Implement a new API for persisted state

[lcwik] Fix flakiness in IntervalBoundedExponentialBackOff

[lcwik] Encode IntervalWindow as upper bound plus duration

[lcwik] Fix DataflowWorkerLoggingHandlerTest on Windows

[davorbonaci] Add encoding id to Coder

[davorbonaci] Windmill.proto changes. Adding cumulative flag to Counter.

[davorbonaci] Upgrade the SDK to use google library versions 1.20.0

[davorbonaci] Proto only changes.

[davorbonaci] Fixing a off-by-one error

[davorbonaci] Adds explicit watermark holds to the Windmill API.

[davorbonaci] SDK return error if user tries to update the same job multiple 
times.

[davorbonaci] Update Windmill API.

[davorbonaci] Add Comparator based versions of Min and Max

[davorbonaci] Java streaming harness support for stable computation ids.

[davorbonaci] Updates Windmill API to support reading watermark holds

[davorbonaci] Replace 'reload' with 'update' terminology

[davorbonaci] Made ByteCount for ShuffleSink consistent with ShuffleSources.

[davorbonaci] Add support for Count.globally().asSingletonView()

[davorbonaci] Use bigdataoss gcs-connecter lib for GCS uploads and reads

[davorbonaci] Improve message for permission errors during resource staging

[davorbonaci] Switch to new Persistent State API

[davorbonaci] Use StateNamespaces with Timers too

[davorbonaci] Minor fixes of documentation.

[davorbonaci] Add a test for aligned and delayed AfterWatermark

[davorbonaci] Fix typo in comment

[davorbonaci] Separate trigger execution from the actual reduction logic

[davorbonaci] Set BigQuery table project ID before validation.

[davorbonaci] Rename WatermarkBag to WatermarkState

[davorbonaci] Implement PaneInfo and populate it during trigger execution

[davorbonaci] Adding dataflow_java_harness_restarts,

[davorbonaci] Track active keys and tokens in StreamingDataflowWorker

[davorbonaci] Expand javadoc for DoFn.ProcessContext.element()

[davorbonaci] Updates to the windmill API

[davorbonaci] Improve performance of BigQueryIO.Write.

[davorbonaci] Roll back this change:

[davorbonaci] Updates to the Windmill API

[davorbonaci] Updates to the Windmill API

[davorbonaci] Examples reorganization

[davorbonaci] Change state_family proto field from string to bytes

[davorbonaci] Support record ids for UnboundedSources

[davorbonaci] More details on expectations of StateInternals.

[davorbonaci] Add WindowFn#getOutputTimestamp

[davorbonaci] Improve performance of BigQueryIO.Write

[davorbonaci] Use WindowFn#getOutputTimestamp to select output times

[davorbonaci] Fix pushback logic in StreamingDataflowWorker

[davorbonaci] Logging improvements for stage and step

[davorbonaci] List suggested properties when parsing command line arguments

[davorbonaci] Add tests for merging the various composite triggers

[davorbonaci] Restore step name context for logging after a DoFn returns 
output()

[davorbonaci] Change state_family proto field back to string

[davorbonaci] Minor test change: adds a user-provided name to ease testing of 
counters

[davorbonaci] Multiplex timers in SDK

[davorbonaci] Remove unneeded ExecutionContext arg to BoundedSource.createReader

[davorbonaci] Count Total Elements as SUM(the windows size for each 
WindowedValue)

[davorbonaci] Prefetch state needed for triggers

[davorbonaci] Fire empty panes if they have new PaneInfo of interest

[davorbonaci] Support UnboundedSource in direct and batch modes

[davorbonaci] Clean up documentation in examples

[davorbonaci] Update version of protobufs used by the SDK

[davorbonaci] Add support for UPDATED job state

[davorbonaci] Remove the Window.Trigger intermediate Builder class

[davorbonaci] Rename SimpleCombineFn to IterableCombineFn

[davorbonaci] Fix NullPointerException in Proto2Coder

[davorbonaci] Work around a type inference change in javac

[davorbonaci] Pass the internal step name to Windmill as a state family

[davorbonaci] Infer the default GCP project from gCloud

[davorbonaci] Remove the SDK defaults for worker configuration

[davorbonaci] Fix View.AsSingleton with default values in streaming

[davorbonaci] Rollback: Remove the SDK defaults for worker configuration

[davorbonaci] Allow specifying each part of a Window operation separately

[davorbonaci] Reduce the per-value cost of SystemReduceFn

[davorbonaci] Add Coder wire format tests

[davorbonaci] Fix a typo in comment

[davorbonaci] Improve types in ShuffleReaderFactoryTest

[davorbonaci] Drop timer firings for windows that have expired

[davorbonaci] Distinguish system StateTags by a one-character prefix

[davorbonaci] Separate ExecutionContext interface from BaseExecutionContext

[davorbonaci] Remove System.out.println from StateSamplerTest

[davorbonaci] Add DoFnRunner.ListOutputManager

[davorbonaci] Update tests to use DoFnRunner.ListOutputManager

[davorbonaci] Use random seed for DirectPipelineRunner unless specified

[davorbonaci] Move GroupByKey validation into validate method

[prabeesh.k] #45 Fix broken links

[davorbonaci] Improve GCS bucket access/missing error messaging

[davorbonaci] DEBUG log level for DirectPipelineRunner random seed

[davorbonaci] Remove cluster manager API selection from SDK.

[davorbonaci] Update special-cases of GroupAlsoByWindow to use 
getOutputTimestamp

[davorbonaci] Fixes two typos in BigQuery validation error messages.

[davorbonaci] Add an example that brings in debugging/testing concepts

[davorbonaci] Adds information about service accounts to source javadocs.

[davorbonaci] Use new watermark hold API.

[davorbonaci] Move PubsubFileInjector to examples.common

[davorbonaci] Remove unneeded methods and type params from DoFnRunner

[davorbonaci] Adds an option to get the index and non-speculative index of a 
Pane

[davorbonaci] Update SDK to correctly set replaceJobId on the Job.

[davorbonaci] Replace "reload" with "update" in SDK.

[davorbonaci] Small bug-fix: Reread after writing the merged accumulator

[davorbonaci] Add support for JSON -> object/map parsing in PipelineOptions

[davorbonaci] Add "-injector" as a suffix to the job name for WindowedWordCount

[davorbonaci] Allow accessing properties of NO_FIRING pane

[davorbonaci] Update exceptions when the job already exists or has been updated

[davorbonaci] Skips publishing empty lines in Pubsub injector

[davorbonaci] Rollback: Add support for JSON -> object/map parsing

[davorbonaci] Fixed the description for --stableUniqueNames

[davorbonaci] Add a Coder<AccumT> to the serialization of CombineFns

[davorbonaci] Remove workerPoolType from pipelineOptions

[davorbonaci] Re-add support for JSON -> object/map parsing

[davorbonaci] Improve the error message of skewed outputWithTimestamp

[davorbonaci] Two trivial fixes to comments in MinimalWordCount.java

[davorbonaci] Improve streaming sdk integration tests

[davorbonaci] WindowedWordCount: cleanup concept numbering

[davorbonaci] Give early errors when using unsupported bounded sources in 
streaming

[davorbonaci] Fixes bugs in hot-key combining

[davorbonaci] Improve choice of number of unbounded source splits

[davorbonaci] DataflowExampleUtils: fix swapped javadoc

[davorbonaci] PackageUtil: warn user on long classpath

[davorbonaci] Updates custom FileBasedSource size estimation.

[davorbonaci] Prints an error when a custom source split exceeds RPC limit.

[davorbonaci] Improve Object overrides in CombineTest

[davorbonaci] Restore: Remove the SDK worker defaults

[davorbonaci] Improve Object overrides in WriteTest

[davorbonaci] Improve Object overrides in CombineValuesTest

[davorbonaci] Clean up GABWViaIteratorsDoFn

[davorbonaci] Fixes bugs in hot-key combining

[davorbonaci] Improve Object overrides for ValueWithRecordId

[davorbonaci] Exposes dataset ID in ApplianceShuffleReader

[davorbonaci] Fix basic Object overrides for KV

[davorbonaci] Add @SafeVarargs to DataflowAssert

[davorbonaci] Add THROUGHPUT_BASED autoscaling algorithm specifier

[davorbonaci] Clean up IntraBundleParallelizationTest

[davorbonaci] Remove a test that overrides API endpoints

[davorbonaci] Mark Coder encoding id methods experimental

[davorbonaci] Add basic PCollectionTupleTest

[davorbonaci] Validates BigQuery datasets in both batch and streaming

[davorbonaci] Improve coder incompatibility error messages

[davorbonaci] De-lint some tests

[davorbonaci] De-lint a few tests

[davorbonaci] Changed Project Name to Project ID in comments.

[davorbonaci] Use a regex to detect potential Project ID problems

[davorbonaci] Internal testing change: change capitalization in an exception

[davorbonaci] Remove experimental annotations from update flags

[davorbonaci] Add a /threadz handler to the batch DataflowWorker's status server

[davorbonaci] AvroCoder: enable @Nullable fields to be deterministic

[davorbonaci] Fix typo in KV.equals() and add tests

[davorbonaci] Fix some Warnings

[davorbonaci] Updating dependency google-api-services-dataflow

[davorbonaci] Add immutability checking to direct evaluation of ParDo

[davorbonaci] Process event time timers before processing time ones

[davorbonaci] Fix some warnings

[davorbonaci] Minor cleanup of ShuffleSinkFactoryTest

[davorbonaci] Replace NoSuchFileException with FileNotFoundException

[davorbonaci] Fix javadoc example for BigQuery#to()

[davorbonaci] Set big query schema in the pipeline option

[davorbonaci] Validate that the GroupByKey inside the View is OK

[davorbonaci] Update SDK to depend on PubSub V1

[davorbonaci] Several fixes for PubsubIO.PubsubReader

[davorbonaci] Minor cleanup of PipelineRunnerTest

[davorbonaci] Do not warn for 404s that are not known to be errors

[davorbonaci] Fix resource not closed warnings

[davorbonaci] De-lint MapTaskExecutorFactoryTest

[davorbonaci] De-lint TransformTreeTest

[davorbonaci] Minor clean up of DataflowWorkProgressUpdaterTest

[davorbonaci] Require getEncodingId() for anonymous CustomCoder subclasses

[davorbonaci] Switch Readers to be abstract classes

[davorbonaci] Do small amount of backoff in worker for UnboundedSources

[davorbonaci] New semantics for empty panes

[davorbonaci] Transparent error messages for workflow job creation failures

[davorbonaci] Makes SourceTestUtils compare structural values

[davorbonaci] Make AfterWatermark.pastEndOfWindow not be a TimeTrigger.

[davorbonaci] Remove dead code from WorkExecutorTest

[davorbonaci] Add @Override to a bunch of methods which should have been tagged

[davorbonaci] Minor clean up of StateSamplerTest

[davorbonaci] Fix some unused code warnings

[davorbonaci] Eliminate warnings in DirectModeExecutionContext

[davorbonaci] TrafficMaxLaneFlow: fix typo

[davorbonaci] Remove warnings about never-thrown declared exceptions

[davorbonaci] Improvements to offset-based sources

[davorbonaci] Clean up maven build

[davorbonaci] Move scripts for automated testing to a directory

[davorbonaci] Attempt to fix Travis working directory

[davorbonaci] Dataflow Travis script: fix relative paths

[davorbonaci] Only pass -Xdoclint:-missing on JDK 8 and newer.

[davorbonaci] Further hardening custom FileBasedSource.

[davorbonaci] Fix NoClassDefFoundError in DebuggingWordCount

[davorbonaci] Reduce state reads for default triggering and discarding

[davorbonaci] Fully Qualify inner type name in DeDupExample

[davorbonaci] Fix PubsubIO name validation

[davorbonaci] Update example READMEs to reflect the examples re-org

[davorbonaci] Update READMEs

[davorbonaci] Update example READMEs to reflect the examples re-org

[davorbonaci] Add link to DataflowJavaSDK-examples

[davorbonaci] Whitespace cleanup

[davorbonaci] Combine the additionalparams values

[davorbonaci] Update Examples README

[davorbonaci] Clarify commandline quoting for DebuggingWordCount

[davorbonaci] Clean up javadoc

[davorbonaci] Dataflow GA: update GitHub's README.md to remove Beta language

[tom] Add contrib module for HadoopFileSource.

[davorbonaci] Added a flag to control the number of threads on a worker

[davorbonaci] Fix DataflowAssert usage in BasicSerializableSourceFormatTest

[davorbonaci] PackageUtil: make directory staging correct, simpler, and fewer 
I/Os

[davorbonaci] Redirect stdout and stderr to a logger in the worker harness

[davorbonaci] DataflowPipelineRunner: fix typo

[davorbonaci] Change how the default value is selected in 
SingletonPCollectionView

[davorbonaci] Remove unnecessary semicolons

[davorbonaci] Properly deactive work when it non-retriably fails

[davorbonaci] Adds package-info for sdk.io.range

[davorbonaci] Fix javadoc formatting with respect to <p> tags

[davorbonaci] Fix default insertion logic in Combine

[davorbonaci] Exclude Windmill Reads from StateSampler

[davorbonaci] Add the starter pipeline and pom as resources for examples

[davorbonaci] Cleanup Javadoc errors

[davorbonaci] Adds counters for implementing shuffle sanity check for Dataflow

[davorbonaci] Add a standalone_examples_pom.xml to the example resources

[davorbonaci] Allow adding PCollections to a PCollectionTuple returned from 
ParDo

[davorbonaci] Refresh progress information after splits

[kmg] Fix documentation of PubsubIO subscription format

[davorbonaci] Added a getter to Write.Bound to retrieve the Sink.

[davorbonaci] SyntheticInput implementation based on custom source

[davorbonaci] Provide default for UnboundedSource.getCurrentRecordId

[davorbonaci] Allow UnboundedSources to report backlog

[davorbonaci] Add null check in ParDo immutability checking

[davorbonaci] Add reset policy to AttemptAndTimeBoundedExponentialBackOff

[davorbonaci] DirectModeExecutionContext extends BaseExecutionContext

[davorbonaci] Add tests for GeneratorUnboundedSource

[davorbonaci] Add an example of trigger usage

[davorbonaci] Export state-sampler information via metrics update.  Also count 
the

[davorbonaci] Minor test cleanup

[davorbonaci] Rename possibly-confusing type variable in WindowedValue

[davorbonaci] Add a boolean thread local to make encodeToByteArray reentrant

[davorbonaci] Adds a bytes-read counter for UnboundedSource for autoscaling

[davorbonaci] Use PropertyNames.OUTPUT string in ParDo translator.

[davorbonaci] Improvements to source testing and file-based sources

[davorbonaci] Fix deps in examples/pom.xml

[davorbonaci] Increase sleep in ReadOperation test to avoid flakiness

[davorbonaci] Change version numbers

[davorbonaci] Create a maven archetype for the starter pipeline

[davorbonaci] Create a Maven Archetype for the word count pipelines

[davorbonaci] Add CoderProvider chaining

[davorbonaci] Fix deps in sdk/pom.xml

[davorbonaci] Configure mvn dependency analysis to ignore runtime deps

[davorbonaci] Make OutputObjectAndByteCounter.countBytes correct, and add tests

[davorbonaci] Remove a few Java warnings

[davorbonaci] Adds counters for bytes written to and read from Windmill

[davorbonaci] BatchModeExecutionContext extends DataflowExecutionContext

[davorbonaci] Do not retain cloned PCollections in DirectPipelineRunner

[davorbonaci] Move streaming worker code to worker package

[davorbonaci] Move zip-related functionality to ZipFiles utility class

[davorbonaci] Add SerializableMatcher and support in DataflowAssert

[davorbonaci] Do not require type @param tags for generic classes

[davorbonaci] TupleTag: minor Javadoc fix and code simplification

[davorbonaci] Add NullableCoder

[davorbonaci] Improve Coder error messages when encoding unsupported null 
objects

[davorbonaci] Fix dangling threads in BigQueryTableInserter

[davorbonaci] Fix Travis build break

[davorbonaci] Provide more explicit error message when Coder inference fails

[davorbonaci] Additional trigger tests

[davorbonaci] Don't include time spent fetching side inputs as user time

[davorbonaci] Fix structuralValue and consistentWithEquals for null

[davorbonaci] TypeDescriptor implements Serializable

[davorbonaci] Fix over-approximating shuffle read counter

[davorbonaci] Improves efficiency of FileBasedSource size estimation.

[davorbonaci] Clean up calls to TableRow.set() in BigQueryTornados

[davorbonaci] Updates dependency google-api-services-dataflow.

[davorbonaci] Minor cleanup of CoGbkResultCoder.equals()

[davorbonaci] MapCoder: use the encoded size hint to allocate the Map

[davorbonaci] Add SetCoder to default Coder registry

[davorbonaci] Allow getFractionConsumed to be called before reading has started

[davorbonaci] Suppress "serial" warnings via pom.xml

[davorbonaci] Add tests for the default implementation of isCompatible

[davorbonaci] Remove unused import

[davorbonaci] Make SerializableMatcher package-private

[davorbonaci] Work around a type inference change in javac

[davorbonaci] Flush invalid entries from UnboundedReader cache

[davorbonaci] Make Filter a proper transform with Coder propagation

[davorbonaci] Instrumenting the worker to get user code msecs

[davorbonaci] Rollback of user code time stats

[davorbonaci] Update ApiSurfaceTest to catch public usage of hamcrest

[davorbonaci] Removes the accidentally left-over SourceTestUtils

[davorbonaci] Updates dependency google-api-services-datastore-protobuf.

[davorbonaci] Version management

[davorbonaci] Removes remaining references to CLOUDSDK_EXTRA_SCOPES

[davorbonaci] Split out parts of AvroIOTest that require Avro-generated classes

[davorbonaci] Updating SDK to use new gcs-connector gcsio/util library.

[davorbonaci] AvroIOTest: test schema upgradability

[davorbonaci] Add Java 8 testing profile with example tests

[davorbonaci] Minor clean up of Partition and PartitionTest

[davorbonaci] Adds dynamic work rebalancing support for ConcatReader.

[davorbonaci] ParDo: cleanup

[davorbonaci] Minor clean up of CoderRegistry

[davorbonaci] Improve some InstanceBuilder error messages

[davorbonaci] Factor out static method-based CoderProvider logic

[davorbonaci] Add FlatMapElements transform

[davorbonaci] Logging upload id on initial upload request at debug level

[davorbonaci] Instrumenting the worker to get user code msecs

[davorbonaci] Enable @DefaultCoder annotations for generic types

[davorbonaci] Add MapElements transform

[davorbonaci] Fix typos in FlatMapElements javadoc

[davorbonaci] Specifies type parameter in View.asX() javadoc examples

[davorbonaci] Make ReaderFactory a real class; remove some reflection

[davorbonaci] Modify GroupAlsoByWindowsDoFn to process batches

[davorbonaci] Demonstrate new lambda-friendly transforms in MinimalWordCount

[davorbonaci] Plumb the WorkItem into StreamingGroupAlsoByWindows

[davorbonaci] Remove interaction logging from trigger tester

[davorbonaci] DatastoreIO: add support for querying namespace, and cleanup

[davorbonaci] Update TriggerTester to submit elements in chunks

[davorbonaci] Plumb PaneInfo through DoFnWithContext.ProcessContext

[davorbonaci] Change the tolerance for the StateSamplerTest.netstingTest.

[davorbonaci] BigQueryIO: validate writeDisposition before writing any data out

[davorbonaci] Remove Google API client versions of Guava classes

[davorbonaci] Re-throw exceptions that happen during cleanup

[davorbonaci] Remove references to existing Run Configurations

[davorbonaci] Explicitly provide slf4j libs by the Dataflow worker

[davorbonaci] Add special handling for triggering in Reshuffle

[davorbonaci] Update StateTag implementations of equals and hashCode

[davorbonaci] Spell correction in the javadoc of WindowFn.

[davorbonaci] Introduce a TriggerBuilder interface

[davorbonaci] Handle exceptions from within the SDK while processing work

[davorbonaci] Version management

[davorbonaci] Removes unnecessary interface SourceFormat

[davorbonaci] Support dynamic work rebalancing for Avro files

[davorbonaci] TriggerExample: fix typo in Javadoc

[davorbonaci] Correct addInput documentation in Combine

[davorbonaci] Remove Combine.globally from batch View.asList() implementation.

[davorbonaci] Remove extraneous serialVersionUID in MapElements

[davorbonaci] Improve default output coder error message in TypedPValue

[davorbonaci] BigQueryIO: minor cleanup

[davorbonaci] Expand error message for erased type variables in CoderRegistry

[davorbonaci] Do not overwrite TypeDescriptor of PCollection in PCollectionTuple

[davorbonaci] FlatMapElements: javadoc fixes

[davorbonaci] Use MapElements in WordCount example

[davorbonaci] Fix comment regarding condition for AfterAll finishing

[davorbonaci] Test simple Combine with Java 8 lambdas and method references

[davorbonaci] BigQueryTableRowIterator: make getF/getV work correctly

[davorbonaci] Hide the CloudDebuggerOptions

[davorbonaci] Avoid deserializing and calling start/finishBundle on empty 
bundles

[davorbonaci] Write: move from transforms to io package

[davorbonaci] Fixes a shutdown race in ReadOperation

[davorbonaci] Enable dynamic work rebalancing when reading from intermediate 
data

[davorbonaci] Add tests using Java 8 method references

[davorbonaci] Add DataflowProfilingOptions

[davorbonaci] Merge before processing elements

[davorbonaci] Add a pipeline option to GcsOptions for user-specified buffer size

[davorbonaci] Fix typo in error message

[davorbonaci] WorkerDiskType option: document and add reference URL

[davorbonaci] Stop sampling after StateSampler.close()

[davorbonaci] Add size-based file rotation to java logger

[davorbonaci] Fix a BigQuery output issue in DirectPipelineRunner

[davorbonaci] Miscellaneous cleanup

[elibixby] Fixed close not being called

[mshytikov] Fixed typo in doc

[mshytikov] Fixed example in BigQueryIO doc

[kevin.sookocheff] Fix BQ docs for pushing to date range

[dhalperi] Return unmodifiable side inputs from the Context

[dhalperi] Change the tolerance for the StateSamplerTest.basicTest and

[dhalperi] Updates dynamic split request handling of FileBasedReader.

[dhalperi] Add a byte limit to the GetWork windmill API call

[dhalperi] Test for AvroCoder for generic type with explicit schema

[dhalperi] Add compact operation to [Keyed]CombineFn

[dhalperi] Byte-limited fetches for TagListState

[dhalperi] PackageUtilTest: make OS/JDK-independent

[dhalperi] Introduce new trigger builder for early/late firings

[dhalperi] Change getByteArray(UTF_8).length to Utf8.encodedBytes

[dhalperi] Fix javadoc compile warnings

[dhalperi] Improve error message during parameterized type inference

[dhalperi] Add external Javadoc links

[dhalperi] Provide a lazy byte size observer for shuffles

[dhalperi] Remove uses of the deprecated getFullNameForTesting

[dhalperi] Set test* values in DirectPipelineRunner from Options

[dhalperi] Adds the ability for sources to specify splittability

[dhalperi] Switch from FluentIterable.of(E[]) to from(asList(E[]))

[dhalperi] Use pass-through WindowFn during Reshuffle

[dhalperi] Move CounterProvider off of BinaryCombine_Fns

[dhalperi] Update pom.xml to the latest version of BigQuery API

[dhalperi] Limited parallelism reporting for TextIO reader.

[dhalperi] Optimize StreamingDataflowWorker progress updates

[dhalperi] Remove Eclipse Starter Project

[dhalperi] Enable Travis container-based infrastructure

[daniel] hadoop contrib: make it compile against latest

[davorbonaci] Adjust the bounds for published parallelism counters.

[davorbonaci] Add dependency-reduced POM to .gitignore

[davorbonaci] Wait until the first sampling event happens to reduce flakyness

[davorbonaci] Update windmill proto library version to 0.4.151029

[davorbonaci] Replace DATA_PARALLEL default job type

[davorbonaci] Adjust ApiSurfaceTest to exclude more extraneous classes

[davorbonaci] Add Test for Empty ParDo with Side Outputs

[davorbonaci] Move MapAggregatorValues to util package

[davorbonaci] Update sdk windmill.proto to match windmill version

[davorbonaci] Add getAggregatorValues to DoFnTester

[davorbonaci] Insert reshards before fixed-shard TextIO sinks

[davorbonaci] Fix minor typos in AfterWatermark javadoc

[davorbonaci] Remove the pipeline option for setting the VM image

[davorbonaci] Update Combine, Top Error Message for Non-global window

[davorbonaci] Memory improvements to the streaming worker

[davorbonaci] Moves the exception throwing to outside the finally block

[davorbonaci] Fix repeat in AfterWatermark.withLateFirings()

[davorbonaci] Adds comment to clarify semantics of the TagList.end_timestamp 
field

[davorbonaci] Prepare windmill.proto for output watermarks

[davorbonaci] Retry setup in DataflowExampleUtils

[davorbonaci] examples/pom.xml: add default slf4j-jdk14 logger

[davorbonaci] Fix continuation of OrFinally trigger

[davorbonaci] Use guava for side input iterable of iterables concatenation

[davorbonaci] Fix formatting in contrib/hadoop to conform to Google rules

[davorbonaci] Avoid double counting bytes read in GroupingShuffleReader

[davorbonaci] Fix formatting in contrib/hadoop to confirm to Google rules

[davorbonaci] Fix a bad merge

[davorbonaci] Update prebuilt proto libraries for Dataflow to 11/9 version

[dano] worker: fix leak in StateSampler

[davorbonaci] Separately test each GroupAlsoByWindows implementation

[davorbonaci] Fix reference to maxReadTime in PubsubIO.Read

[davorbonaci] Testing improvements regarding custom sources in streaming

[davorbonaci] Output watermark should be specified per key rather then per

[davorbonaci] Improve UncaughtExceptionHandler

[davorbonaci] Integration test for shuffle sanity check for Dataflow

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Improve Dataflow javadocs

[davorbonaci] Fix some grammar in Dataflow documentation

[davorbonaci] Minor fixes to trigger javadoc

[davorbonaci] Updates to windowing javadoc

[davorbonaci] Touchup documentation in Runners Package

[davorbonaci] Paginate list results from Windmill to Worker

[davorbonaci] Javadoc cleanup

[davorbonaci] Add WithTimestamps PTransform

[davorbonaci] DataflowPipelineWorkerPoolOptions: add TEARDOWN_ON_SUCCESS option

[davorbonaci] Move version numbering into parent pom

[davorbonaci] Removes unused ReaderIterator.copy

[davorbonaci] Insert reshards before fixed-shard AvroIO sinks

[davorbonaci] Merges FileBasedReader into TextReader

[davorbonaci] Adds Protocol Buffer definition for gRPC communication with 
Windmill

[davorbonaci] Improve javadoc for View transforms

[davorbonaci] Fix Javadoc typo

[davorbonaci] AvroIO: improve Javadoc

[davorbonaci] Reduce the usage of stateByName() in MTE and adding more tests.

[davorbonaci] AvroSource: improve Javadoc

[davorbonaci] Javadoc improvements

[davorbonaci] Update Dataflow to use 11/17 version of prebuilt proto library

[davorbonaci] Fix typo in CoGbkResult message

[davorbonaci] Upgrade to bigdataoss library 1.4.3

[davorbonaci] BigQueryIO: improve Javadoc

[davorbonaci] PubSubIO: improve Javadoc

[davorbonaci] Fix WithTimestamps Java 8 Example

[davorbonaci] Deprecate the incorrect chaining in RetryHttpRequestInitializer

[davorbonaci] Extend input type of Flatten#iterables

[davorbonaci] Stream Iterable<T> state through MergedBag

[davorbonaci] Condense PipelineOptionsFactory Exceptions

[davorbonaci] TextIO: improve javadoc

[davorbonaci] Make the IterableLikeCoder efficient for many small values

[davorbonaci] Fire a pane on AfterAll merge if it was not finished in all 
windows

[davorbonaci] Fix a worker crash when writing to intermediate files

[davorbonaci] Enable application default credentials for Cloud Shell

[davorbonaci] Merge all subtriggers in AfterAll.

[davorbonaci] PubSubIO: add, test, document support for RFC3339 timestamps

[davorbonaci] Add customized output timestamp for GroupByKey result

[davorbonaci] Minor javadoc updates.

[davorbonaci] Javadoc cleanup to BoundedSource

[davorbonaci] Cleanup Javadoc on BlockBasedSource

[davorbonaci] Migrate to use nano time and a nano sleep in tests

[davorbonaci] Cleanup Javadoc on CompressedSource

[davorbonaci] Convert OutputTimeFn from interface to abstract class

[davorbonaci] Remove Unneccessary import in CombineJava8Test

[davorbonaci] Fix sizing of windowed side inputs to improve caching

[davorbonaci] Improve Lambda Support in WithKeys, RemoveDuplicates

[davorbonaci] Setup data structure to allow for overrides for batch pipelines

[davorbonaci] DataflowPipelineOptions: use gcloud project default if available

[davorbonaci] Filter: fixup uses of #by and #byPredicate

[davorbonaci] ByteArrayCoder: remove unused import, fix Javadoc link

[davorbonaci] Coders: fix Javadoc

[davorbonaci] Coder: fix javadoc

[davorbonaci] BlockBasedSource: fix javadoc links

[davorbonaci] Fix issue with SerializableCoder and JAXBCoder to not close stream

[davorbonaci] Occasionally compact combined state on write.

[davorbonaci] Version management

[davorbonaci] Improvements to DataflowWorkProgressUpdaterTest

[davorbonaci] Reduce availability of StepContext#getExecutionContext

[davorbonaci] Fixed a race in StateSampler.

[davorbonaci] Genericize BaseExecutionContext

[davorbonaci] Fixes a null pointer in SourceOperationExecutor.

[davorbonaci] Remove NullPointerException in DataflowWorkProgressUpdaterTest

[davorbonaci] Refine LATE pane semantics and implementation

[davorbonaci] Use 8MB page sizes to match minimums used elsewhere

[davorbonaci] Updates google-api-services-dataflow dependency

[davorbonaci] Return a future for WindmillState.persist()

[davorbonaci] Minor javadoc updates.

[davorbonaci] CombineTest: remove unused import, suppress a Java warning

[davorbonaci] Rename TriggerTester to ReduceFnTester

[davorbonaci] Fixes a data race in OffsetBasedSource

[davorbonaci] A series of batch and streaming pipeline examples

[davorbonaci] Uses the new progress/split request classes

[davorbonaci] Update OffsetBasedSource javadoc

[davorbonaci] Improve error messaging when applying unsupported transform in 
batch

[davorbonaci] Drop data only for expired windows

[davorbonaci] Do not require .withAllowedLateness for GlobalWindows

[davorbonaci] Remove Trigger.OnElementContext.element()

[davorbonaci] Change the assembly of state keys to be more efficient

[davorbonaci] Improve javadoc for Pipeline, PipelineResult

[hildrum] fix typo

[lcwik] Remove jetty-jmx dependency

[lcwik] Small javadoc fixes for DoFn and friends

[lcwik] Add Proto2Coder as universal fallback

[lcwik] Add dependency on jetty-servlet

[lcwik] CountingSource: bounded/unbounded source of longs

[lcwik] Have Dataflow internally depend on Guava 19.0

[lcwik] Add a scalable bloom filter implementation

[lcwik] Touch up ParDo javadoc

[lcwik] Avoid integer overflow when computing maximum cache weight

[lcwik] Enable state caching for per-key windmill state

[lcwik] DataflowPipelineRunnerTest: make pass if user has default project set

[lcwik] Touch up KV javadoc

[lcwik] RetryHttpRequestInitializer: verify that SocketTimeoutExceptions are

[lcwik] Touch up javadoc for values package

[lcwik] Add more tests to DoFnTesterTest

[lcwik] Add OAuth2 addresses to GcpOptions

[lcwik] Make ParDo.withSideInputs cumulative

[lcwik] Tidy some worker code

[lcwik] DataflowWorkProgressUpdaterTest: fix data races in the test itself

[lcwik] Tidy some utility code

[lcwik] Update the overview page for the javadoc

[lcwik] Ensures that BoundedSource and UnboundedSource are Serializable.

[lcwik] Always pass maxNumWorkers in job settings

[lcwik] Factor AvroUtils into a separate class

[lcwik] Change the 'gaming' injector credentials check logic

[lcwik] Increment environment major version to 4

[lcwik] Tidy PCollectionViews

[lcwik] Tidy some type variables

[lcwik] Tidy some tests and support code

[lcwik] Setting table creation definition in WindowedWordCount example

[lcwik] Add ByteCoder

[lcwik] Eagerly merge new windows into existing windows

[lcwik] Native reader/sink for sorted key/value data

[lcwik] FlatMapElementsJava8Test: fix a typo

[lcwik] Add support for testing timestamps in DoFnTester

[lcwik] Read bounded PubSubIO in examples pipelines

[lcwik] Rollback state caching.

[lcwik] Decouple TriggerContextFactory from ReduceFn somewhat

[lcwik] Migrate from deprecated isAssignableFrom to equivalent isSupertypeOf.

[lcwik] Rollback of read bounded PubSubIO in examples pipelines.

[lcwik] Fix AfterAll merge logic

[lcwik] Run OutputTimeFn-related tests on the service and local runner

[lcwik] Decreases size of input for AvroSourceTest

[lcwik] Decouple TriggerContext and friends from ReduceFn.Timers

[lcwik] XmlSource: clarify requirement for unique keys in Javadoc

[lcwik] Resubmit: Read bounded PubSubIO in examples pipelines

[lcwik] Fix BigQueryIO.Read to work the same in Direct and Dataflow runners

[lcwik] CombineFnWithContext SDK API and worker code

[lcwik] Upgrade google-api-services-dataflow to v1b3-rev14-1.21.0

[lcwik] Roll forward state caching

[lcwik] Cleanup a comment

[lcwik] Implement a utility to check dynamic work rebalancing for a source.

[lcwik] DatastoreWordCount: use an ancestor query to ensure consistent results

[lcwik] Makes ReaderIterator have the same interface as Source.Reader

[lcwik] Rename TriggerExecutorTest to ReduceFnRunnerTest

[lcwik] Fix Maven warning by specifying version of the build-helper plugin

[lcwik] Break out TriggerTester from ReduceFnTester

[lcwik] DatastoreIO: support user-provided limits on number of results

[lcwik] Bound the size of commit queues in the streaming worker

[lcwik] Fix data races in BigQueryTableInserter and BigQueryUtilTest

[lcwik] Fix javadoc @link warnings

[lcwik] Adding "DocInclude" metadata comments to the "game" example

[lcwik] Adding worker ID to the upload id logging

[lcwik] DataflowAssert: throw when .equals(Object) is called

[lcwik] Generalize the 'game' example BigQuery write classes

[davorbonaci] Use .jar for staged directory packages

[davorbonaci] DefaultProjectFactory: make it use new gcloud properties files

[davorbonaci] Checkstyle: support disabling specific analyzers

[davorbonaci] Add README.md for the "game" example series

[lcwik] Stops holding initializationStateLock while opening the reader

[lcwik] Upgrade JaCoCo to 0.7.5

[lcwik] Fix typo: wrong table column name

[lcwik] Fix typo in OutputTimeFn

[lcwik] Implement typeFromId(DatabindContext,String) within CoderUtils

[lcwik] Fixes a bug in custom unbounded readers

[M.Runesson] Adapt to be able to upload to maven-central

[M.Runesson] Aditional info to simplify release.

[hildrum] fix typo

[M.Runesson] Fix review comments.

[davorbonaci] Version management

[davorbonaci] BigQueryTableRowIterator: elide columns with null values

[davorbonaci] Rollback "BigQueryTableRowIterator: elide columns with null 
values"

[davorbonaci] Deterministically choose freshest aggregations in pipeline results

[davorbonaci] Split streaming status pages into servlets

[davorbonaci] Expose dependent realtime watermark via Windmill protos

[davorbonaci] Add ByteStringCoder, a coder for ByteStrings

[davorbonaci] CustomSources: remove dead code

[davorbonaci] Ignore Eclipse project files in root .gitignore

[davorbonaci] BigQueryTableRowIterator: elide columns with null values

[davorbonaci] Split out StatusDataProviders

[davorbonaci] Proto changes for multi-worker support.

[davorbonaci] Use the standard set of status pages for Batch

[davorbonaci] Add FinishedTriggers abstraction with BitSet and Set 
implementations

[davorbonaci] CustomSources: add logs and normalize log levels

[davorbonaci] SingletonAssert: add a notEqualTo matcher

[davorbonaci] Implement Counter#merge

[davorbonaci] Optimize mergeAccumulators by reusing an existing accumulator

[davorbonaci] Handle when Dataflow service tells us that values are sorted in 
GBK

[davorbonaci] Continues unifying ReaderIterator and Source.Reader

[davorbonaci] Fix Coder.Context equality and hashCode

[davorbonaci] Update prebuilt proto libraries for Dataflow to 2016-01-20 version

[davorbonaci] Upgrade Jackson dependency from 2.4.5 to 2.7.0

[davorbonaci] Expose event time and synchronized upstream processing time

[davorbonaci] Update Triggers to new shouldFire() based semantics

[davorbonaci] StreamingWriteFn: check if table exists before creating

[davorbonaci] Let shuffle reader and writer update counters.

[davorbonaci] Provide a reasonable default to Window.Bound#apply

[davorbonaci] Allow 'game' examples to append to existing BigQuery tables.

[davorbonaci] Create StreamingGroupAlsoByWindowsDoFnRunner

[davorbonaci] DatastoreIO: do not split when QuerySplitter fails

[davorbonaci] BigQueryTableInserter: retry rateLimitExceeded API calls

[davorbonaci] AvroCoder: more efficient use of Avro APIs

[davorbonaci] Modify GameStats window definitions for blog post

[davorbonaci] Add new property names

[davorbonaci] Add increment support with positive infinity

[davorbonaci] BigQueryTableRowIterator: handle query failures

[davorbonaci] Add CounterSet#merge(CounterSet)

[davorbonaci] Allow unflattened results from a BigQuery query-based export

[davorbonaci] Clean up GameStats session branch for blog post

[davorbonaci] Account for per-window overhead in WindmillStateCache

[davorbonaci] Add support to configure the shuffle client library with a 
property.

[davorbonaci] Create worker maven module

[davorbonaci] Add explicit record type for Ism files

[davorbonaci] Upgrade protobuf runtime to version 3.0.0-beta-1

[davorbonaci] Refactor LateDataDroppingDoFnRunner

[davorbonaci] BigQueryIO: remove DirectRunner dependence on Dataflow native 
reader

[davorbonaci] runners.CountingSource: rename to TestCountingSource

[davorbonaci] Validate DoFn.createAggregate is not called during pipeline 
processing

[davorbonaci] DataflowPipelineRunner: retry source splitting when too many 
bundles

[davorbonaci] DataflowWorker: elide some for loops for unused log levels

[davorbonaci] BigQueryReader: simplify and remove some redundant code

[davorbonaci] Create internal-only classifier for Dataflow SDK

[rav] Change BQ mode to append instead of truncate

[dhalperi] Disable surefire forking in Travis-CI

[davorbonaci] Move Dataflow worker mains to worker maven module

[davorbonaci] Removes dependency on LegacyReaderIterator from more readers

[davorbonaci] Add support to export BigQuery files in Avro format

[davorbonaci] Move some status pages to worker module

[davorbonaci] CombineWithContext support in batch PartialGroupByKeyOperation

[davorbonaci] Fix TimeTrigger#isCompatible

[davorbonaci] Move StructuralByteArray to the coder package

[davorbonaci] AvroUtils: default missing field schema mode to NULLABLE, and test

[davorbonaci] Add ByteCount for BigQueryIO streaming write

[davorbonaci] Default to exporting BigQuery files in Avro format

[davorbonaci] Add InProcessPipelineRunner, Evaluation Interfaces

[davorbonaci] Upgrade maven shade plugin to avoid infinite loop

[davorbonaci] Add InMemoryWatermarkManager

[davorbonaci] Have worker maven module depend on unshaded/unrelocated test jar

[davorbonaci] Separate CustomSources into worker and non-worker pieces

[davorbonaci] Most methods in com.google.common will soon have @CheckReturnValue

[davorbonaci] Move ReaderFactory.Registry to ReaderRegistry

[davorbonaci] Add Ism file format version 2

[davorbonaci] Always explicitly pass ReaderRegistry to MapTaskExecutorFactory

[davorbonaci] Move worker tests to worker maven module

[davorbonaci] CustomSourcesTest: use a smaller byte limit to reduce test runtime

[davorbonaci] Explicitly pass ReaderFactory from worker classes to 
SideInputUtils

[davorbonaci] Move MapTaskExecutorFactory and system DoFns and their factories

[davorbonaci] Move several operations and dependencies to worker module

[davorbonaci] Split InMemoryBundle into Read/Write only interfaces

[davorbonaci] ApiSurfaceTest: whitelist com.google.auth package

[davorbonaci] Make SinkFactory a real class; remove some reflection

[davorbonaci] Marks getAllowedTimestampSkew as deprecated.

[davorbonaci] Add the transforms required to create an Ism side input

[davorbonaci] Don't wrap system DoFn exceptions.

[davorbonaci] Move BatchModeExecutionContext and dependents to worker module

[davorbonaci] Handle SynchronizedProcessingTime in InMemoryWatermarkManager

[davorbonaci] Move streaming worker code into worker module

[davorbonaci] Move Windmill-specific code and dependents to worker module

[davorbonaci] Move Reader classes and dependents to worker module

[davorbonaci] Move over IO sink factories and dependants

[davorbonaci] Render the /threadz page using plain text.

[davorbonaci] Move remainder of status servlets

[davorbonaci] Decouple StateContext from ReduceFn

[davorbonaci] Allows specifying if an OffsetBasedSource supports dynamic 
splitting

[davorbonaci] Create a top level GroupingTable interface and its factory class

[davorbonaci] Experimental Cloud Bigtable sink

[davorbonaci] Tighten BigtableIO's API surface exposure

[davorbonaci] Separate Timers interface from ReduceFn

[davorbonaci] Merge TimeTrigger into its only subclass

[davorbonaci] ApiSurface: add debug messages

[davorbonaci] Migrate AvroIO to internally use Read.from(AvroSource)

[davorbonaci] Upgrade to SLF4J 1.7.14 so that we can use slf4j-api MDC

[davorbonaci] Eagerly merge all window state

[davorbonaci] Add the Ism side input reader

[davorbonaci] Move over various worker classes to worker maven module

[davorbonaci] Move over several shuffle classes to the worker maven module

[davorbonaci] Add a DoFnRunner for StreamingGroupAlsoByWindows with side inputs

[davorbonaci] Handle Timers in InMemoryWatermarkManager

[davorbonaci] Move over worker logging classes to worker maven module

[davorbonaci] Move worker implementation classes of aggregators to worker maven 
module

[davorbonaci] Remove Base64Utils which is dead code

[davorbonaci] Move over several Dataflow worker specific classes to the worker 
maven

[davorbonaci] ByteKey: a key represented byte[]

[davorbonaci] Remove duplicate definition of MetricUpdate utilities

[davorbonaci] Enable new Ism side input format by default for batch Dataflow 
jobs

[davorbonaci] Use shaded guava from within worker maven module

[davorbonaci] ByteKey: add a static instance EMPTY for the empty key

[davorbonaci] Move miscellaneous worker classes to worker maven module

[davorbonaci] Allow StreamingDataflowWorker to run against a remote gRPC 
windmill

[davorbonaci] Move CustomSources logic to live in more appropriate places

[davorbonaci] Add filename-based compression selection to CompressedSource

[davorbonaci] Make internal state APIs explicitly keyed

[davorbonaci] Add KeyedCombiningValueStateTag containing a KeyedCombineFn

[davorbonaci] Explicitly select GroupAlsoByWindow in default GBK

[davorbonaci] Add ByteKeyRange and ByteKeyRangeTracker

[davorbonaci] Make some ReduceFnRunner implementation details private

[davorbonaci] Treat ReduceFn and Trigger exceptions as system exceptions

[davorbonaci] Decouple NonEmptyPanes from ReduceFn somewhat

[davorbonaci] Decouple TriggerRunner from ReduceFn

[davorbonaci] Support reading uncompressed files when gzip is expected

[davorbonaci] Explicitly provide StateInternals to ReduceFnRunner

[davorbonaci] Fix typo in TimerInternals comment.

[davorbonaci] Fix propagation of streaming keys

[davorbonaci] Do not assume channel is nonempty in TextIO gzip detection

[davorbonaci] Make WatermarkHold package-private

[davorbonaci] Minor revision of ReduceFnRunner docs

[davorbonaci] Minor revision of ActiveWindowSet javadoc

[davorbonaci] Make empty flatten work in streaming

[davorbonaci] Minor revision of TriggerRunner javadoc

[davorbonaci] Improve IntraBundleParallelizationTest

[davorbonaci] Move specialized GroupAlsoByWindow implementations

[davorbonaci] Declare SideInputs on Combine.GroupedValues

[davorbonaci] Update Dataflow API version to v1b3-rev19-1.21.0

[davorbonaci] Add InProcess Clock abstraction, nanoTime implementation

[davorbonaci] Improve TypeDescriptor inference of SimpleFunction

[davorbonaci] Fix AfterSynchronizedProcessingTime.java

[davorbonaci] Remove debug statement from test

[davorbonaci] Make sure every hold is accompanied by a proximate timer

[davorbonaci] Add support for splitting a compressed source for uncompressed 
files

[davorbonaci] Add --worker_harness_container_image PipelineOption

[davorbonaci] Use new keyed state API to remove ReduceFn.Factory

[davorbonaci] Implement CopyOnAccessInMemoryStateInternals

[davorbonaci] Handle null input bundles in InMemoryWatermarkManager

[davorbonaci] Remove unneccessary type from GroupAlsoByWindowViaWindowSet

[davorbonaci] Never drop late data in Reshuffle

[davorbonaci] BigtableIO: add a bounded source for Google Cloud Bigtable

[davorbonaci] ApproximateUnique[Test]: improve efficiency and cleanup test

[davorbonaci] Refactor to use try-with-resources

[davorbonaci] StateSamplerTest: fewer repeats per reused counter name

[davorbonaci] Add InProcessSideInputContainer

[davorbonaci] Update FileIOChannelFactory.expand() to only return file resources

[davorbonaci] Add validation for single objects in GcsUtil.expand

[davorbonaci] Rename StateContext to StateAccessor

[davorbonaci] ByteKeyRange: parameterize expensive combinatorial tests

[davorbonaci] Migrate TextIO.Read to be a custom source

[davorbonaci] Make earliest Watermark State available in CopyOnAccessState

[davorbonaci] Add ForwardingPTransform

[davorbonaci] Move test for post-GBK timestamp ordering to worker module

[davorbonaci] Move DataflowWorkerLoggingMDC to the worker module

[davorbonaci] Rolling back since this breaks tests

[davorbonaci] Clear empty CopyOnAccessInMemoryStateInternals

[davorbonaci] Add Timers and State to InProcessTransformResult

[davorbonaci] Migrate TextIO.Read to be a custom source

[davorbonaci] Remove InProcessPipelineRunner evaluator package

[davorbonaci] Proto2Coder: fix static modifier check

[davorbonaci] Proto2Coder: minor cleanup

[davorbonaci] Add InProcessTimerInternals

[davorbonaci] Move VerifyDynamicWorkRebalancing and dependents to worker module

[davorbonaci] Allow TransformEvaluatorFactory#forApplication to throw

[davorbonaci] Add TypeDescriptors to Primitive PTransforms in Tests

[davorbonaci] Remove native Dataflow text reader

[davorbonaci] Encode elements in InProcessCreate

[davorbonaci] Expose base output file name on FileBasedSink

[davorbonaci] Move over Google Cloud Dataflow worker utilities to worker module

[davorbonaci] Add new test proto messages that use map fields

[davorbonaci] Migrate AvroIO.Write to a custom sink

[davorbonaci] Move Google Cloud Dataflow worker utilities to worker module

[davorbonaci] Port state to new, future-free API

[davorbonaci] Move some worker-and-example-only dependencies out of sdk

[davorbonaci] Change visibility of FileBasedSource subclass methods and fix 
return

[davorbonaci] Make TestPipeline slightly less DataflowPipelineRunner-centric

[davorbonaci] Add used-but-undeclared dependency on google-http-client

[davorbonaci] Migrate TextIO.Write to a custom sink

[davorbonaci] Move Google Cloud Dataflow native sinks to worker module

[davorbonaci] Reverts "Move Google Cloud Dataflow native sinks to worker module"

[davorbonaci] Add the slf4j-jdk bridge to the integration tests

[davorbonaci] Revert "Migrate TextIO.Write to a custom sink"

[davorbonaci] Add KeyedWorkItemCoder

[davorbonaci] Revert "Migrate AvroIO.Write to a custom sink"

[davorbonaci] Resubmit "Migrate AvroIO.Write to a custom sink"

[davorbonaci] Add GroupByKey InProcess override

[davorbonaci] Proto2Coder: recompute the extension registry when mutated

[davorbonaci] Support CombineFnWithContext in GroupAlsoByWindows

[davorbonaci] Add InProcess Override for CreatePCollectionView

[davorbonaci] Update maven-dependency-plugin to latest

[davorbonaci] ProtoCoder: a Coder for Protocol Buffers Messages

[davorbonaci] Add used-but-undeclared findbugs JSR305 dependencies

[davorbonaci] Handle PCollectionList.empty() in FlattenEvaluatorFactory

[davorbonaci] Set worker harness container image to INVALID until next release

[davorbonaci] Handle Undeclared Side Outputs in ParDoInProcessEvaluator

[davorbonaci] Rollback revert "Migrate TextIO.Write to a custom sink"

[davorbonaci] Update Timers and State in the InProcess ParDoEvaluator

[davorbonaci] Rollback reverts "Move Google Cloud Dataflow native sinks to 
worker

[davorbonaci] Use a static variable for CoderCalled in WriteTest

[davorbonaci] Honor user requested shard limits for AvroIO.Write on

[davorbonaci] Adjust dependencies to avoid pulling in unneeded stax-api

[davorbonaci] Handle multiple requests in InProcess Read Primitives

[davorbonaci] Finish Flattenning InProcess package

[davorbonaci] Ensure a TypedPValue has a Coder on finishSpecifying

[davorbonaci] Switch to the start state when lazily initializing

[davorbonaci] Fix SDK deps and enable strict enforcement

[davorbonaci] Fix worker dependencies and turn on strict checking

[davorbonaci] Update worker harness container image

[fjp] Revert "Add a first README.md file (at least to trigger the github

[fjp] Update README for initial code drop.

[fjp] Remove Google-specific contribution rules

------------------------------------------
[...truncated 1803 lines...]
jdk1.8.0_66/jre/lib/amd64/libnet.so
jdk1.8.0_66/jre/lib/amd64/libjavafx_font.so
jdk1.8.0_66/jre/lib/amd64/libprism_common.so
jdk1.8.0_66/jre/lib/amd64/libnio.so
jdk1.8.0_66/jre/lib/amd64/libprism_es2.so
jdk1.8.0_66/jre/lib/amd64/libinstrument.so
jdk1.8.0_66/jre/lib/amd64/libkcms.so
jdk1.8.0_66/jre/lib/amd64/libawt_xawt.so
jdk1.8.0_66/jre/lib/amd64/libmanagement.so
jdk1.8.0_66/jre/lib/amd64/libunpack.so
jdk1.8.0_66/jre/lib/amd64/libgstreamer-lite.so
jdk1.8.0_66/jre/lib/amd64/libawt_headless.so
jdk1.8.0_66/jre/lib/amd64/libsplashscreen.so
jdk1.8.0_66/jre/lib/fontconfig.properties.src
jdk1.8.0_66/jre/lib/psfont.properties.ja
jdk1.8.0_66/jre/lib/fontconfig.Turbo.properties.src
jdk1.8.0_66/jre/lib/jce.jar
jdk1.8.0_66/jre/lib/flavormap.properties
jdk1.8.0_66/jre/lib/jfxswt.jar
jdk1.8.0_66/jre/lib/fontconfig.SuSE.10.properties.src
jdk1.8.0_66/jre/lib/fontconfig.SuSE.11.bfc
jdk1.8.0_66/jre/COPYRIGHT
jdk1.8.0_66/jre/THIRDPARTYLICENSEREADME-JAVAFX.txt
jdk1.8.0_66/jre/Welcome.html
jdk1.8.0_66/jre/README
jdk1.8.0_66/README.html
Building remotely on jenkins-ubuntu-1404-4gb-407 (jenkins-cloud-4GB cloud-slave 
Ubuntu ubuntu) in workspace <https://builds.apache.org/job/beam/ws/>
Cloning the remote Git repository
Cloning repository https://git-wip-us.apache.org/repos/asf/incubator-beam.git
 > git init <https://builds.apache.org/job/beam/ws/> # timeout=10
Fetching upstream changes from 
https://git-wip-us.apache.org/repos/asf/incubator-beam.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/incubator-beam.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/incubator-beam.git # timeout=10
 > git config --add remote.origin.fetch +refs/heads/*:refs/remotes/origin/* # 
 > timeout=10
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/incubator-beam.git # timeout=10
Fetching upstream changes from 
https://git-wip-us.apache.org/repos/asf/incubator-beam.git
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/incubator-beam.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/master^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/master^{commit} # timeout=10
Checking out Revision 211e76abf9ba34c35ef13cca279cbeefdad7c406 
(refs/remotes/origin/master)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 211e76abf9ba34c35ef13cca279cbeefdad7c406
 > git rev-list 11e842717f70298a4ea8436363b3101117685f60 # timeout=10
Parsing POMs
Discovered a new module 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent Google Cloud 
Dataflow Java SDK - Parent
Discovered a new module 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-all Google Cloud 
Dataflow Java SDK - All
Discovered a new module 
com.google.cloud.dataflow:google-cloud-dataflow-java-examples-all Google Cloud 
Dataflow Java Examples - All
Discovered a new module 
com.google.cloud.dataflow:google-cloud-dataflow-java-archetypes-starter Google 
Cloud Dataflow Java SDK - Starter Archetype
Discovered a new module 
com.google.cloud.dataflow:google-cloud-dataflow-java-archetypes-examples Google 
Cloud Dataflow Java SDK - Examples Archetype
Modules changed, recalculating dependency graph
Established TCP socket on 60521
maven32-agent.jar already up to date
maven32-interceptor.jar already up to date
maven3-interceptor-commons.jar already up to date
[beam] $ /jenkins/tools/hudson.model.JDK/jdk1.8.0_66/bin/java -Xmx2g -Xms256m 
-XX:MaxPermSize=512m -cp 
/jenkins/maven32-agent.jar:/jenkins/tools/hudson.tasks.Maven_MavenInstallation/maven-3.3.3/boot/plexus-classworlds-2.5.2.jar:/jenkins/tools/hudson.tasks.Maven_MavenInstallation/maven-3.3.3/conf/logging
 jenkins.maven3.agent.Maven32Main 
/jenkins/tools/hudson.tasks.Maven_MavenInstallation/maven-3.3.3 
/jenkins/slave.jar /jenkins/maven32-interceptor.jar 
/jenkins/maven3-interceptor-commons.jar 60521
Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=512m; 
support was removed in 8.0
<===[JENKINS REMOTING CAPACITY]===>   channel started
Executing Maven:  -B -f <https://builds.apache.org/job/beam/ws/pom.xml> 
-Dmaven.repo.local=/jenkins/maven-repositories/0 -B -e clean deploy
[INFO] Error stacktraces are turned on.
[INFO] Scanning for projects...
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/archetype-packaging/2.4/archetype-packaging-2.4.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/archetype-packaging/2.4/archetype-packaging-2.4.pom
 (2 KB at 3.0 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/maven-archetype/2.4/maven-archetype-2.4.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/maven-archetype/2.4/maven-archetype-2.4.pom
 (13 KB at 429.7 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/archetype-packaging/2.4/archetype-packaging-2.4.jar
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-utils/1.1/plexus-utils-1.1.jar
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/archetype/archetype-packaging/2.4/archetype-packaging-2.4.jar
 (8 KB at 122.6 KB/sec)
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-utils/1.1/plexus-utils-1.1.jar
 (165 KB at 2032.3 KB/sec)
[INFO] ------------------------------------------------------------------------
[INFO] Reactor Build Order:
[INFO] 
[INFO] Google Cloud Dataflow Java SDK - Parent
[INFO] Google Cloud Dataflow Java SDK - All
[INFO] Google Cloud Dataflow Java Examples - All
[INFO] Google Cloud Dataflow Java SDK - Starter Archetype
[INFO] Google Cloud Dataflow Java SDK - Examples Archetype
[INFO]                                                                         
[INFO] ------------------------------------------------------------------------
[INFO] Building Google Cloud Dataflow Java SDK - Parent 1.5.0-SNAPSHOT
[INFO] ------------------------------------------------------------------------
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-install-plugin/2.4/maven-install-plugin-2.4.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-install-plugin/2.4/maven-install-plugin-2.4.pom
 (7 KB at 259.6 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-install-plugin/2.4/maven-install-plugin-2.4.jar
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-install-plugin/2.4/maven-install-plugin-2.4.jar
 (27 KB at 1052.8 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-deploy-plugin/2.7/maven-deploy-plugin-2.7.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-deploy-plugin/2.7/maven-deploy-plugin-2.7.pom
 (6 KB at 238.3 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-deploy-plugin/2.7/maven-deploy-plugin-2.7.jar
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/apache/maven/plugins/maven-deploy-plugin/2.7/maven-deploy-plugin-2.7.jar
 (27 KB at 1048.1 KB/sec)
[INFO] 
[INFO] --- maven-clean-plugin:2.5:clean (default-clean) @ 
google-cloud-dataflow-java-sdk-parent ---
[INFO] 
[INFO] --- maven-install-plugin:2.4:install (default-install) @ 
google-cloud-dataflow-java-sdk-parent ---
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-digest/1.0/plexus-digest-1.0.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-digest/1.0/plexus-digest-1.0.pom
 (2 KB at 45.0 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-components/1.1.7/plexus-components-1.1.7.pom
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-components/1.1.7/plexus-components-1.1.7.pom
 (5 KB at 220.8 KB/sec)
[INFO] Downloading: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-digest/1.0/plexus-digest-1.0.jar
[INFO] Downloaded: 
https://repo.maven.apache.org/maven2/org/codehaus/plexus/plexus-digest/1.0/plexus-digest-1.0.jar
 (12 KB at 482.5 KB/sec)
[INFO] Installing <https://builds.apache.org/job/beam/ws/pom.xml> to 
/jenkins/maven-repositories/0/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-SNAPSHOT.pom
[INFO] 
[INFO] --- maven-deploy-plugin:2.7:deploy (default-deploy) @ 
google-cloud-dataflow-java-sdk-parent ---
[INFO] Downloading: 
http://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/maven-metadata.xml
[INFO] Uploading: 
http://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom
[INFO] ------------------------------------------------------------------------
[INFO] Reactor Summary:
[INFO] 
[INFO] Google Cloud Dataflow Java SDK - Parent ............ FAILURE [  2.604 s]
[INFO] Google Cloud Dataflow Java SDK - All ............... SKIPPED
[INFO] Google Cloud Dataflow Java Examples - All .......... SKIPPED
[INFO] Google Cloud Dataflow Java SDK - Starter Archetype . SKIPPED
[INFO] Google Cloud Dataflow Java SDK - Examples Archetype  SKIPPED
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 7.562 s
[INFO] Finished at: 2016-02-26T23:29:32+00:00
[INFO] Final Memory: 13M/286M
[INFO] ------------------------------------------------------------------------
[ERROR] Failed to execute goal 
org.apache.maven.plugins:maven-deploy-plugin:2.7:deploy (default-deploy) on 
project google-cloud-dataflow-java-sdk-parent: Failed to deploy artifacts: 
Could not transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized. -> [Help 1]
org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute goal 
org.apache.maven.plugins:maven-deploy-plugin:2.7:deploy (default-deploy) on 
project google-cloud-dataflow-java-sdk-parent: Failed to deploy artifacts: 
Could not transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:216)
        at 
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
        at 
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
        at 
org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:116)
        at 
org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:80)
        at 
org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)
        at 
org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:128)
        at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:307)
        at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:193)
        at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:106)
        at 
org.jvnet.hudson.maven3.launcher.Maven32Launcher.main(Maven32Launcher.java:132)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.codehaus.plexus.classworlds.launcher.Launcher.launchStandard(Launcher.java:330)
        at 
org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:238)
        at jenkins.maven3.agent.Maven32Main.launch(Maven32Main.java:186)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at hudson.maven.Maven3Builder.call(Maven3Builder.java:136)
        at hudson.maven.Maven3Builder.call(Maven3Builder.java:71)
        at hudson.remoting.UserRequest.perform(UserRequest.java:120)
        at hudson.remoting.UserRequest.perform(UserRequest.java:48)
        at hudson.remoting.Request$2.run(Request.java:326)
        at 
hudson.remoting.InterceptingExecutorService$1.call(InterceptingExecutorService.java:68)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.maven.plugin.MojoExecutionException: Failed to deploy 
artifacts: Could not transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.apache.maven.plugin.deploy.DeployMojo.execute(DeployMojo.java:193)
        at 
org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134)
        at 
org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208)
        ... 31 more
Caused by: org.apache.maven.artifact.deployer.ArtifactDeploymentException: 
Failed to deploy artifacts: Could not transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.apache.maven.artifact.deployer.DefaultArtifactDeployer.deploy(DefaultArtifactDeployer.java:143)
        at 
org.apache.maven.plugin.deploy.AbstractDeployMojo.deploy(AbstractDeployMojo.java:167)
        at 
org.apache.maven.plugin.deploy.DeployMojo.execute(DeployMojo.java:149)
        ... 33 more
Caused by: org.eclipse.aether.deployment.DeploymentException: Failed to deploy 
artifacts: Could not transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.eclipse.aether.internal.impl.DefaultDeployer.deploy(DefaultDeployer.java:317)
        at 
org.eclipse.aether.internal.impl.DefaultDeployer.deploy(DefaultDeployer.java:245)
        at 
org.eclipse.aether.internal.impl.DefaultRepositorySystem.deploy(DefaultRepositorySystem.java:413)
        at 
org.apache.maven.artifact.deployer.DefaultArtifactDeployer.deploy(DefaultArtifactDeployer.java:139)
        ... 35 more
Caused by: org.eclipse.aether.transfer.ArtifactTransferException: Could not 
transfer artifact 
com.google.cloud.dataflow:google-cloud-dataflow-java-sdk-parent:pom:1.5.0-20160226.232928-1
 from/to google-snapshots 
(http://oss.sonatype.org/content/repositories/google-snapshots/): Failed to 
transfer file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.eclipse.aether.connector.basic.ArtifactTransportListener.transferFailed(ArtifactTransportListener.java:43)
        at 
org.eclipse.aether.connector.basic.BasicRepositoryConnector$TaskRunner.run(BasicRepositoryConnector.java:355)
        at 
org.eclipse.aether.connector.basic.BasicRepositoryConnector.put(BasicRepositoryConnector.java:274)
        at 
org.eclipse.aether.internal.impl.DefaultDeployer.deploy(DefaultDeployer.java:311)
        ... 38 more
Caused by: org.apache.maven.wagon.TransferFailedException: Failed to transfer 
file: 
https://oss.sonatype.org/content/repositories/google-snapshots/com/google/cloud/dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-20160226.232928-1.pom.
 Return code is: 401, ReasonPhrase: Unauthorized.
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:627)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:541)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:613)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:541)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:523)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:517)
        at 
org.apache.maven.wagon.providers.http.AbstractHttpClientWagon.put(AbstractHttpClientWagon.java:497)
        at 
org.eclipse.aether.transport.wagon.WagonTransporter$PutTaskRunner.run(WagonTransporter.java:644)
        at 
org.eclipse.aether.transport.wagon.WagonTransporter.execute(WagonTransporter.java:427)
        at 
org.eclipse.aether.transport.wagon.WagonTransporter.put(WagonTransporter.java:410)
        at 
org.eclipse.aether.connector.basic.BasicRepositoryConnector$PutTaskRunner.runTask(BasicRepositoryConnector.java:510)
        at 
org.eclipse.aether.connector.basic.BasicRepositoryConnector$TaskRunner.run(BasicRepositoryConnector.java:350)
        ... 40 more
[ERROR] 
[ERROR] Re-run Maven using the -X switch to enable full debug logging.
[ERROR] 
[ERROR] For more information about the errors and possible solutions, please 
read the following articles:
[ERROR] [Help 1] 
http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException
[JENKINS] Archiving 
<https://builds.apache.org/job/beam/ws/maven-archetypes/starter/pom.xml> to 
com.google.cloud.dataflow/google-cloud-dataflow-java-archetypes-starter/1.5.0-SNAPSHOT/google-cloud-dataflow-java-archetypes-starter-1.5.0-SNAPSHOT.pom
No prior successful build to compare, so performing full copy of artifacts
[JENKINS] Archiving <https://builds.apache.org/job/beam/ws/pom.xml> to 
com.google.cloud.dataflow/google-cloud-dataflow-java-sdk-parent/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-parent-1.5.0-SNAPSHOT.pom
No prior successful build to compare, so performing full copy of artifacts
[JENKINS] Archiving <https://builds.apache.org/job/beam/ws/examples/pom.xml> to 
com.google.cloud.dataflow/google-cloud-dataflow-java-examples-all/1.5.0-SNAPSHOT/google-cloud-dataflow-java-examples-all-1.5.0-SNAPSHOT.pom
No prior successful build to compare, so performing full copy of artifacts
[JENKINS] Archiving <https://builds.apache.org/job/beam/ws/sdk/pom.xml> to 
com.google.cloud.dataflow/google-cloud-dataflow-java-sdk-all/1.5.0-SNAPSHOT/google-cloud-dataflow-java-sdk-all-1.5.0-SNAPSHOT.pom
No prior successful build to compare, so performing full copy of artifacts
[JENKINS] Archiving 
<https://builds.apache.org/job/beam/ws/maven-archetypes/examples/pom.xml> to 
com.google.cloud.dataflow/google-cloud-dataflow-java-archetypes-examples/1.5.0-SNAPSHOT/google-cloud-dataflow-java-archetypes-examples-1.5.0-SNAPSHOT.pom
No prior successful build to compare, so performing full copy of artifacts
ERROR: Unable to send to address: 
a...@mygrid.org.uk@bf327186-88b3-11dd-a302-d386e5130c1c
javax.mail.internet.AddressException: Domain contains illegal character in 
string ``a...@mygrid.org.uk@bf327186-88b3-11dd-a302-d386e5130c1c''
        at 
javax.mail.internet.InternetAddress.checkAddress(InternetAddress.java:1269)
        at javax.mail.internet.InternetAddress.parse(InternetAddress.java:1091)
        at javax.mail.internet.InternetAddress.parse(InternetAddress.java:633)
        at javax.mail.internet.InternetAddress.<init>(InternetAddress.java:111)
        at hudson.tasks.Mailer.stringToAddress(Mailer.java:206)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.toNormalizedAddress(MimeMessageBuilder.java:262)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.addRecipients(MimeMessageBuilder.java:141)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.addRecipients(MimeMessageBuilder.java:133)
        at hudson.tasks.MailSender.createEmptyMail(MailSender.java:388)
        at hudson.tasks.MailSender.createFailureMail(MailSender.java:262)
        at hudson.tasks.MailSender.createMail(MailSender.java:180)
        at hudson.tasks.MailSender.run(MailSender.java:109)
        at hudson.tasks.MailSender.execute(MailSender.java:103)
        at hudson.maven.reporters.MavenMailer.end(MavenMailer.java:74)
        at hudson.maven.AbstractMavenBuilder.end(AbstractMavenBuilder.java:104)
        at 
hudson.maven.MavenModuleSetBuild$MavenModuleSetBuildExecution.doRun(MavenModuleSetBuild.java:859)
        at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:534)
        at hudson.model.Run.execute(Run.java:1738)
        at hudson.maven.MavenModuleSetBuild.run(MavenModuleSetBuild.java:531)
        at hudson.model.ResourceController.execute(ResourceController.java:98)
        at hudson.model.Executor.run(Executor.java:410)
Sending e-mails to: commits@beam.incubator.apache.org t...@analytical-labs.com 
dhalp...@google.com f...@google.com lc...@google.com m.runes...@gmail.com 
davorbon...@users.noreply.github.com dan...@pocock.pro
channel stopped
ERROR: Unable to send to address: 
a...@mygrid.org.uk@bf327186-88b3-11dd-a302-d386e5130c1c
javax.mail.internet.AddressException: Domain contains illegal character in 
string ``a...@mygrid.org.uk@bf327186-88b3-11dd-a302-d386e5130c1c''
        at 
javax.mail.internet.InternetAddress.checkAddress(InternetAddress.java:1269)
        at javax.mail.internet.InternetAddress.parse(InternetAddress.java:1091)
        at javax.mail.internet.InternetAddress.parse(InternetAddress.java:633)
        at javax.mail.internet.InternetAddress.<init>(InternetAddress.java:111)
        at hudson.tasks.Mailer.stringToAddress(Mailer.java:206)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.toNormalizedAddress(MimeMessageBuilder.java:262)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.addRecipients(MimeMessageBuilder.java:141)
        at 
jenkins.plugins.mailer.tasks.MimeMessageBuilder.addRecipients(MimeMessageBuilder.java:133)
        at hudson.tasks.MailSender.createEmptyMail(MailSender.java:388)
        at hudson.tasks.MailSender.createFailureMail(MailSender.java:262)
        at hudson.tasks.MailSender.createMail(MailSender.java:180)
        at hudson.tasks.MailSender.run(MailSender.java:109)
        at hudson.tasks.MailSender.execute(MailSender.java:103)
        at 
hudson.maven.MavenModuleSetBuild$MavenModuleSetBuildExecution.cleanUp(MavenModuleSetBuild.java:1065)
        at hudson.model.Run.execute(Run.java:1785)
        at hudson.maven.MavenModuleSetBuild.run(MavenModuleSetBuild.java:531)
        at hudson.model.ResourceController.execute(ResourceController.java:98)
        at hudson.model.Executor.run(Executor.java:410)

Reply via email to