From acf2ea3c3b3dabc23c474a94a65d42b5cebd5046 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Tue, 15 May 2018 19:59:53 +0200 Subject: [PATCH 1/3] [BEAM-3902] Fix license headers and remove unused poms. --- sdks/java/extensions/euphoria/README.md | 21 +- .../euphoria/euphoria-beam/build.gradle | 18 ++ .../extensions/euphoria/euphoria-beam/pom.xml | 117 -------- .../cz/seznam/euphoria/beam/BeamExecutor.java | 12 +- .../euphoria/beam/BeamExecutorContext.java | 12 +- .../cz/seznam/euphoria/beam/BeamFlow.java | 12 +- .../seznam/euphoria/beam/DoFnCollector.java | 12 +- .../euphoria/beam/FlatMapTranslator.java | 12 +- .../seznam/euphoria/beam/FlowTranslator.java | 12 +- .../seznam/euphoria/beam/InputTranslator.java | 12 +- .../beam/LazyAccumulatorProvider.java | 12 +- .../euphoria/beam/OperatorTranslator.java | 12 +- .../euphoria/beam/ReduceByKeyTranslator.java | 12 +- .../beam/ReduceStateByKeyTranslator.java | 12 +- .../euphoria/beam/SingleValueCollector.java | 12 +- .../cz/seznam/euphoria/beam/TypeUtils.java | 12 +- .../seznam/euphoria/beam/UnionTranslator.java | 12 +- .../beam/WrappedPCollectionOperator.java | 12 +- .../seznam/euphoria/beam/coder/PairCoder.java | 12 +- .../euphoria/beam/io/BeamBoundedSource.java | 12 +- .../euphoria/beam/io/BeamUnboundedSource.java | 12 +- .../euphoria/beam/io/BeamWriteSink.java | 12 +- .../cz/seznam/euphoria/beam/io/KryoCoder.java | 12 +- .../euphoria/beam/window/BeamWindow.java | 12 +- .../euphoria/beam/window/BeamWindowFn.java | 12 +- .../beam/window/BeamWindowedElement.java | 12 +- .../cz/seznam/euphoria/beam/BeamFlowTest.java | 12 +- .../cz/seznam/euphoria/beam/FlatMapTest.java | 12 +- .../seznam/euphoria/beam/ReduceByKeyTest.java | 12 +- .../beam/testkit/BeamExecutorProvider.java | 12 +- .../beam/testkit/BeamOperatorsSuite.java | 12 +- .../beam/testkit/BeamOperatorsTest.java | 12 +- .../src/test/resources/log4j.properties | 7 - .../euphoria/euphoria-core/build.gradle | 18 ++ .../extensions/euphoria/euphoria-core/pom.xml | 76 ----- .../core/annotation/audience/Audience.java | 12 +- .../core/annotation/operator/Basic.java | 12 +- .../core/annotation/operator/Derived.java | 12 +- .../core/annotation/operator/Recommended.java | 12 +- .../annotation/operator/StateComplexity.java | 12 +- .../annotation/stability/Experimental.java | 12 +- .../core/client/accumulators/Accumulator.java | 12 +- .../accumulators/AccumulatorProvider.java | 12 +- .../core/client/accumulators/Counter.java | 12 +- .../core/client/accumulators/Histogram.java | 12 +- .../core/client/accumulators/Timer.java | 12 +- .../accumulators/VoidAccumulatorProvider.java | 12 +- .../euphoria/core/client/dataset/Dataset.java | 12 +- .../core/client/dataset/Datasets.java | 12 +- .../core/client/dataset/InputDataset.java | 12 +- .../core/client/dataset/OutputDataset.java | 12 +- .../core/client/dataset/windowing/Count.java | 12 +- .../dataset/windowing/GlobalWindowing.java | 12 +- .../dataset/windowing/MergingWindowing.java | 12 +- .../client/dataset/windowing/Session.java | 12 +- .../core/client/dataset/windowing/Time.java | 12 +- .../dataset/windowing/TimeInterval.java | 12 +- .../client/dataset/windowing/TimeSliding.java | 12 +- .../core/client/dataset/windowing/Window.java | 12 +- .../dataset/windowing/WindowedElement.java | 12 +- .../client/dataset/windowing/Windowing.java | 12 +- .../euphoria/core/client/flow/Flow.java | 12 +- .../euphoria/core/client/flow/Util.java | 12 +- .../client/functional/BinaryFunction.java | 12 +- .../core/client/functional/BinaryFunctor.java | 12 +- .../functional/CombinableBinaryFunction.java | 12 +- .../functional/CombinableReduceFunction.java | 12 +- .../functional/CompositeUnaryFunction.java | 12 +- .../core/client/functional/Consumer.java | 12 +- .../client/functional/ExtractEventTime.java | 12 +- .../client/functional/ReduceFunction.java | 12 +- .../core/client/functional/ReduceFunctor.java | 12 +- .../functional/SameArgsBinaryFunction.java | 12 +- .../client/functional/TernaryFunction.java | 12 +- .../core/client/functional/TypeHintAware.java | 12 +- .../core/client/functional/UnaryFunction.java | 12 +- .../client/functional/UnaryFunctionEnv.java | 12 +- .../core/client/functional/UnaryFunctor.java | 12 +- .../client/functional/UnaryPredicate.java | 12 +- .../core/client/functional/VoidFunction.java | 12 +- .../core/client/io/BoundedDataSource.java | 12 +- .../core/client/io/BoundedReader.java | 12 +- .../core/client/io/CloseableIterator.java | 12 +- .../euphoria/core/client/io/Collector.java | 12 +- .../euphoria/core/client/io/Context.java | 12 +- .../euphoria/core/client/io/DataSink.java | 12 +- .../euphoria/core/client/io/DataSinks.java | 12 +- .../euphoria/core/client/io/DataSource.java | 12 +- .../euphoria/core/client/io/Environment.java | 12 +- .../core/client/io/ExternalIterable.java | 12 +- .../euphoria/core/client/io/ListDataSink.java | 12 +- .../core/client/io/ListDataSource.java | 12 +- .../core/client/io/MultiDataSink.java | 12 +- .../euphoria/core/client/io/SpillTools.java | 12 +- .../euphoria/core/client/io/StdoutSink.java | 12 +- .../core/client/io/UnboundedDataSource.java | 12 +- .../core/client/io/UnboundedPartition.java | 12 +- .../core/client/io/UnboundedReader.java | 12 +- .../client/io/UnsplittableBoundedSource.java | 12 +- .../euphoria/core/client/io/VoidSink.java | 12 +- .../euphoria/core/client/io/Writer.java | 12 +- .../euphoria/core/client/lib/Split.java | 12 +- .../core/client/operator/AssignEventTime.java | 12 +- .../core/client/operator/Builders.java | 12 +- .../core/client/operator/CountByKey.java | 12 +- .../core/client/operator/Distinct.java | 12 +- .../client/operator/ElementWiseOperator.java | 12 +- .../euphoria/core/client/operator/Filter.java | 12 +- .../core/client/operator/FlatMap.java | 12 +- .../core/client/operator/FullJoin.java | 12 +- .../euphoria/core/client/operator/Join.java | 12 +- .../core/client/operator/LeftJoin.java | 12 +- .../core/client/operator/MapElements.java | 12 +- .../core/client/operator/Operator.java | 12 +- .../operator/OptionalMethodBuilder.java | 12 +- .../core/client/operator/ReduceByKey.java | 12 +- .../client/operator/ReduceStateByKey.java | 12 +- .../core/client/operator/ReduceWindow.java | 12 +- .../core/client/operator/RightJoin.java | 12 +- .../client/operator/SingleInputOperator.java | 12 +- .../core/client/operator/StateAware.java | 12 +- .../StateAwareWindowWiseOperator.java | 12 +- ...ateAwareWindowWiseSingleInputOperator.java | 12 +- .../core/client/operator/StateSupport.java | 12 +- .../core/client/operator/SumByKey.java | 12 +- .../core/client/operator/TopPerKey.java | 12 +- .../euphoria/core/client/operator/Union.java | 12 +- .../core/client/operator/WindowAware.java | 12 +- .../client/operator/WindowWiseOperator.java | 12 +- .../operator/WindowingRequiredException.java | 12 +- .../client/operator/hint/ComputationHint.java | 12 +- .../core/client/operator/hint/OutputHint.java | 12 +- .../core/client/operator/hint/SizeHint.java | 12 +- .../client/operator/state/ListStorage.java | 12 +- .../operator/state/ListStorageDescriptor.java | 12 +- .../state/MergingStorageDescriptor.java | 12 +- .../core/client/operator/state/State.java | 12 +- .../client/operator/state/StateContext.java | 12 +- .../client/operator/state/StateFactory.java | 12 +- .../client/operator/state/StateMerger.java | 12 +- .../core/client/operator/state/Storage.java | 12 +- .../operator/state/StorageDescriptor.java | 12 +- .../operator/state/StorageProvider.java | 12 +- .../client/operator/state/ValueStorage.java | 12 +- .../state/ValueStorageDescriptor.java | 12 +- .../triggers/AfterFirstCompositeTrigger.java | 12 +- .../core/client/triggers/CountTrigger.java | 12 +- .../core/client/triggers/NoopTrigger.java | 12 +- .../client/triggers/PeriodicTimeTrigger.java | 12 +- .../core/client/triggers/TimeTrigger.java | 12 +- .../core/client/triggers/Trigger.java | 12 +- .../core/client/triggers/TriggerContext.java | 12 +- .../core/client/type/AbstractTypeAware.java | 12 +- .../client/type/TypeAwareReduceFunctor.java | 12 +- .../client/type/TypeAwareUnaryFunction.java | 12 +- .../client/type/TypeAwareUnaryFunctor.java | 12 +- .../euphoria/core/client/type/TypeHint.java | 12 +- .../euphoria/core/client/util/Either.java | 12 +- .../euphoria/core/client/util/Fold.java | 12 +- .../seznam/euphoria/core/client/util/Max.java | 12 +- .../euphoria/core/client/util/Pair.java | 12 +- .../euphoria/core/client/util/Sums.java | 12 +- .../euphoria/core/client/util/Triple.java | 12 +- .../core/executor/AbstractExecutor.java | 12 +- .../euphoria/core/executor/Constants.java | 12 +- .../euphoria/core/executor/Executor.java | 12 +- .../euphoria/core/executor/FlowUnfolder.java | 12 +- .../euphoria/core/executor/FlowValidator.java | 12 +- .../euphoria/core/executor/VectorClock.java | 12 +- .../euphoria/core/executor/graph/DAG.java | 12 +- .../euphoria/core/executor/graph/Node.java | 12 +- .../core/executor/greduce/GroupReducer.java | 12 +- .../core/executor/greduce/TimerSupport.java | 12 +- .../core/executor/greduce/TriggerStorage.java | 12 +- .../executor/io/FsSpillingListStorage.java | 12 +- .../core/executor/io/GenericSpillTools.java | 12 +- .../core/executor/io/SerializerFactory.java | 12 +- .../core/executor/io/SpillFileFactory.java | 12 +- .../executor/util/InMemExternalIterable.java | 12 +- .../core/executor/util/MultiValueContext.java | 12 +- .../executor/util/OperatorTranslator.java | 12 +- .../executor/util/SingleValueContext.java | 12 +- .../seznam/euphoria/core/time/Scheduler.java | 12 +- .../euphoria/core/time/TimeProvider.java | 12 +- .../euphoria/core/time/TimeProviderAware.java | 12 +- .../euphoria/core/time/TimeProviders.java | 12 +- .../euphoria/core/time/TimerScheduler.java | 12 +- .../euphoria/core/util/ExceptionUtils.java | 12 +- .../cz/seznam/euphoria/core/util/IOUtils.java | 12 +- .../euphoria/core/util/InstanceUtils.java | 12 +- .../seznam/euphoria/core/util/Settings.java | 12 +- .../client/dataset/windowing/SessionTest.java | 12 +- .../dataset/windowing/TimeSlidingTest.java | 12 +- .../client/dataset/windowing/TimeTest.java | 12 +- .../dataset/windowing/TimestampedElement.java | 12 +- .../dataset/windowing/WindowingTest.java | 12 +- .../euphoria/core/client/flow/TestFlow.java | 12 +- .../euphoria/core/client/io/EmptyReader.java | 12 +- .../core/client/io/ListDataSinkTest.java | 12 +- .../core/client/io/MockStreamDataSource.java | 12 +- .../core/client/io/MultiDataSinkTest.java | 12 +- .../euphoria/core/client/lib/SplitTest.java | 12 +- .../core/client/operator/CountByKeyTest.java | 12 +- .../core/client/operator/DistinctTest.java | 12 +- .../core/client/operator/FilterTest.java | 12 +- .../core/client/operator/FlatMapTest.java | 12 +- .../core/client/operator/HintTest.java | 12 +- .../core/client/operator/JoinTest.java | 12 +- .../core/client/operator/MapElementsTest.java | 12 +- .../core/client/operator/ReduceByKeyTest.java | 12 +- .../client/operator/ReduceStateByKeyTest.java | 12 +- .../client/operator/ReduceWindowTest.java | 12 +- .../core/client/operator/SumByKeyTest.java | 12 +- .../core/client/operator/TopPerKeyTest.java | 12 +- .../core/client/operator/UnionTest.java | 12 +- .../euphoria/core/client/operator/Util.java | 12 +- .../core/client/util/IOUtilsTest.java | 12 +- .../core/executor/FlowUnfolderTest.java | 12 +- .../euphoria/core/executor/graph/DAGTest.java | 12 +- .../io/FsSpillingListStorageTest.java | 12 +- .../executor/io/GenericSpillToolsTest.java | 12 +- .../executor/io/JavaSerializationFactory.java | 12 +- .../io/TmpFolderSpillFileFactory.java | 12 +- .../euphoria/core/testing/DatasetAssert.java | 12 +- .../euphoria/core/time/TimeProvidersTest.java | 12 +- .../euphoria/core/util/SettingsTest.java | 12 +- .../src/test/resources/logback-test.xml | 21 -- .../euphoria/euphoria-examples/build.gradle | 32 --- .../euphoria/euphoria-examples/pom.xml | 140 --------- .../seznam/euphoria/examples/Executors.java | 127 --------- .../euphoria/examples/HBaseStreamer.java | 106 ------- .../seznam/euphoria/examples/HFileLoader.java | 157 ----------- .../cz/seznam/euphoria/examples/Utils.java | 164 ----------- .../examples/wordcount/AccessLogCount.java | 266 ------------------ .../examples/wordcount/SimpleWordCount.java | 213 -------------- .../src/main/resources/log4j.properties | 7 - .../wordcount/SimpleWordCountTest.java | 84 ------ .../src/test/resources/avro/data.avro | Bin 272 -> 0 bytes .../euphoria/euphoria-fluent/build.gradle | 18 ++ .../euphoria/euphoria-fluent/pom.xml | 62 ---- .../cz/seznam/euphoria/fluent/Dataset.java | 12 +- .../java/cz/seznam/euphoria/fluent/Flow.java | 12 +- .../cz/seznam/euphoria/fluent/Fluent.java | 12 +- .../cz/seznam/euphoria/fluent/FluentTest.java | 12 +- .../euphoria/euphoria-local/build.gradle | 18 ++ .../euphoria/euphoria-local/pom.xml | 70 ----- .../local/AbstractTriggerScheduler.java | 12 +- .../executor/local/AttachedWindowing.java | 12 +- .../euphoria/executor/local/Collector.java | 12 +- .../seznam/euphoria/executor/local/Datum.java | 12 +- .../euphoria/executor/local/ExecPath.java | 12 +- .../euphoria/executor/local/ExecUnit.java | 12 +- .../euphoria/executor/local/KeyedWindow.java | 12 +- .../executor/local/LocalExecutor.java | 12 +- .../executor/local/LocalSpillTools.java | 12 +- .../executor/local/LocalStateContext.java | 12 +- .../executor/local/LocalStorageProvider.java | 12 +- .../executor/local/NoopTriggerScheduler.java | 12 +- .../local/ProcessingTimeTriggerScheduler.java | 12 +- .../local/ReduceStateByKeyReducer.java | 12 +- .../executor/local/TriggerScheduler.java | 12 +- .../euphoria/executor/local/Triggerable.java | 12 +- .../executor/local/WatermarkEmitStrategy.java | 12 +- .../local/WatermarkTriggerScheduler.java | 12 +- .../local/WindowedElementCollector.java | 12 +- .../executor/local/BasicOperatorTest.java | 12 +- .../executor/local/JoinOperatorTest.java | 12 +- .../executor/local/LocalExecutorTest.java | 12 +- .../seznam/euphoria/executor/local/Util.java | 12 +- .../executor/local/VectorClockTest.java | 12 +- .../executor/local/WindowingTest.java | 12 +- .../local/testkit/LocalExecutorProvider.java | 12 +- .../local/testkit/LocalOperatorTest.java | 12 +- .../euphoria-operator-testkit/build.gradle | 18 ++ .../euphoria-operator-testkit/pom.xml | 96 ------- .../operator/test/AllOperatorsSuite.java | 12 +- .../operator/test/BroadcastHashJoinTest.java | 12 +- .../operator/test/CountByKeyTest.java | 12 +- .../euphoria/operator/test/DistinctTest.java | 12 +- .../euphoria/operator/test/FilterTest.java | 12 +- .../euphoria/operator/test/FlatMapTest.java | 12 +- .../euphoria/operator/test/IntWindow.java | 12 +- .../euphoria/operator/test/JoinTest.java | 12 +- .../test/JoinWindowEnforcementTest.java | 12 +- .../operator/test/MapElementsTest.java | 12 +- .../operator/test/ReduceByKeyTest.java | 12 +- .../operator/test/ReduceStateByKeyTest.java | 12 +- .../operator/test/ReduceWindowTest.java | 12 +- .../euphoria/operator/test/SinkTest.java | 12 +- .../euphoria/operator/test/SumByKeyTest.java | 12 +- .../euphoria/operator/test/TopPerKeyTest.java | 12 +- .../euphoria/operator/test/UnionTest.java | 12 +- .../seznam/euphoria/operator/test/Util.java | 12 +- .../euphoria/operator/test/WatermarkTest.java | 12 +- .../euphoria/operator/test/WindowingTest.java | 12 +- .../test/accumulators/LongCounter.java | 12 +- .../test/accumulators/LongHistogram.java | 12 +- .../test/accumulators/NanosecondTimer.java | 12 +- .../SingleJvmAccumulatorProvider.java | 12 +- .../test/accumulators/SnapshotProvider.java | 12 +- .../test/accumulators/Snapshotable.java | 12 +- .../test/junit/AbstractOperatorTest.java | 12 +- .../test/junit/ExecutorEnvironment.java | 12 +- .../operator/test/junit/ExecutorProvider.java | 12 +- .../test/junit/ExecutorProviderRunner.java | 12 +- .../operator/test/junit/Processing.java | 12 +- .../euphoria/euphoria-testing/build.gradle | 18 ++ .../euphoria/euphoria-testing/pom.xml | 64 ----- .../euphoria/testing/AbstractFlowTest.java | 12 +- .../euphoria/testing/DatasetAssert.java | 12 +- 310 files changed, 2115 insertions(+), 3231 deletions(-) delete mode 100644 sdks/java/extensions/euphoria/euphoria-beam/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/test/resources/log4j.properties delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/test/resources/logback-test.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/build.gradle delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Executors.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HBaseStreamer.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HFileLoader.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Utils.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/AccessLogCount.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCount.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/main/resources/log4j.properties delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/test/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCountTest.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-examples/src/test/resources/avro/data.avro delete mode 100644 sdks/java/extensions/euphoria/euphoria-fluent/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-local/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-operator-testkit/pom.xml delete mode 100644 sdks/java/extensions/euphoria/euphoria-testing/pom.xml diff --git a/sdks/java/extensions/euphoria/README.md b/sdks/java/extensions/euphoria/README.md index aa6bbe47ffbb9..887d8f2e63bbe 100644 --- a/sdks/java/extensions/euphoria/README.md +++ b/sdks/java/extensions/euphoria/README.md @@ -1,6 +1,23 @@ -# Euphoria + -[![Build Status](https://travis-ci.org/seznam/euphoria.svg?branch=master)](https://travis-ci.org/seznam/euphoria) +# Euphoria Euphoria is an open source Java API for creating unified big-data processing flows. It provides an engine independent programming model diff --git a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle index 5473642ef13f3..2d5cb36e453ef 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature() diff --git a/sdks/java/extensions/euphoria/euphoria-beam/pom.xml b/sdks/java/extensions/euphoria/euphoria-beam/pom.xml deleted file mode 100644 index d391e492c70d5..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-beam/pom.xml +++ /dev/null @@ -1,117 +0,0 @@ - - - - - 4.0.0 - - cz.seznam.euphoria - euphoria-parent - 0.9-SNAPSHOT - ../pom.xml - - - - 2.4.0 - - - euphoria-beam - jar - - ${project.groupId}:${project.artifactId} - - Executor implementation for Apache Beam. - - - - - ${project.groupId} - euphoria-core - - - - com.google.code.findbugs - jsr305 - - - - org.slf4j - slf4j-api - - - - org.apache.beam - beam-sdks-java-core - ${beam.version} - - - - com.esotericsoftware.kryo - kryo - - - - cz.seznam.euphoria - euphoria-operator-testkit - test - - - - cz.seznam.euphoria - euphoria-testing - test - - - - org.apache.beam - beam-runners-direct-java - ${beam.version} - - - - org.slf4j - slf4j-log4j12 - 1.7.25 - test - - - - junit - junit - test - - - - org.hamcrest - hamcrest-all - 1.3 - test - - - - com.google.code.findbugs - annotations - 3.0.1u2 - provided - - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java index 97524a80767a2..fca3d4424a535 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java index 592981762ec01..69aed8e080bc6 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java index 397c3eae2a22f..e769401c02aff 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java index 3c84848e1db3d..f396755a5a8cd 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java index 2e4f6ef63b402..c794107932830 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java index c93eadf327559..b18f35d47a536 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java index cded0dd20c949..5a76677dbb27e 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java index 88fbfc126538f..2ea4901998956 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java index 0c7a6dea7b7be..f60f367830fb1 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java index ec459e052981f..d78b066f765fb 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java index 3899f02f0af49..47ffb0eeb268b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java index feb59c7472922..c65e8df546f19 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java index cbded9d84d2c8..1bd6163fa97fe 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java index d2a08241932a9..5b3c129616071 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java index f0534e52e370e..85bb2df1ab62b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java index 527650a040348..4cc5e16326f04 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java index 0b947103d28a0..35d0a43371e51 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java index daa7f9311196d..80bef5d35f9d3 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java index 28ed2cc50db0e..f0a64a2d8ad30 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java index d91b82b418240..4bc925c14a91a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java index 716574c5b080c..a85d8c9b973a5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java index 8c3589cc71420..649f29ddf320b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java index e98bdad9bd8fd..3f48e29f96c8a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java index cf09fde51b61a..6db2c07c64600 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java index e8d89646dc134..0c4049b9e4a69 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java index 6c9758c07cfaa..116676cadda37 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java index 9bb875a31f03a..94c865175c07f 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java index 11a95aa0f3592..6deed8a99790d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java index c800aa753fd9c..8021ca628983d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/resources/log4j.properties b/sdks/java/extensions/euphoria/euphoria-beam/src/test/resources/log4j.properties deleted file mode 100644 index 341c290fd85bc..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/resources/log4j.properties +++ /dev/null @@ -1,7 +0,0 @@ -log4j.rootCategory=INFO, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} [%t] %p %c: %m%n -log4j.logger.cz.seznam.euphoria=INFO, console -log4j.additivity.cz.seznam.euphoria=false diff --git a/sdks/java/extensions/euphoria/euphoria-core/build.gradle b/sdks/java/extensions/euphoria/euphoria-core/build.gradle index 318b482e84ba8..aec8b36e15518 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-core/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature(enableFindbugs: false) diff --git a/sdks/java/extensions/euphoria/euphoria-core/pom.xml b/sdks/java/extensions/euphoria/euphoria-core/pom.xml deleted file mode 100644 index 5379d42a19641..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/pom.xml +++ /dev/null @@ -1,76 +0,0 @@ - - - - - 4.0.0 - - cz.seznam.euphoria - euphoria-parent - 0.9-SNAPSHOT - ../pom.xml - - - euphoria-core - jar - - ${project.groupId}:${project.artifactId} - - Euphoria's Core provides the API to isolate client side programs from - specific execution engine implementations. - - - - - - cz.seznam.euphoria - thirdparty-guava-${guava.version} - - - - com.google.code.findbugs - jsr305 - - - - org.slf4j - slf4j-api - - - - junit - junit - test - - - - org.mockito - mockito-all - test - - - - ch.qos.logback - logback-classic - test - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java index 821b4a6b9fe4c..874d989033c51 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java index afb9556260306..43ba65f65c9a7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java index fcb1fc9068939..c9fe25b299bf8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java index 7514d7936a6bc..e4a96d8ecd2a9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java index 42278046a65ea..4bd9b6419da82 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java index f943389f6abaf..bcd57a1073acb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java index 49cd104260c2c..95104409244ab 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java index 170b71595c645..377af190a35a4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java index 28ed9716e18a1..25da368aff42a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java index f6b30bc5ecfc3..0c345c34c5143 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java index 02a7fb42a9d51..be439c11f2807 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java index 7d591977887d1..fe29bd0809760 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java index c456a09a24961..e87a4f5e029b5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java index 77e58f53a9249..67f399e09d847 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java index 201251030392a..9c5e24c1e73ba 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java index 867033f0dbf3a..55b1cdf50157f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java index 362dd3d99e342..156e17c96addd 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java index 85ee164a6ee35..7174c702b7e99 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java index 82a4a488c0563..2db906ff8e2c3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java index 14c4517983e02..121bf277b0c25 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java index ae0493a2fcc6a..70be1d36d7928 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java index d64c96942c029..3e93b40f0b875 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java index 87fb464620867..e5269a82ebf0c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java index 02a9fc6ffdce5..53ca8d7544cb1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java index 8f75af4e68f4c..95a5fec2f9b17 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java index 20f12d0686d97..0a2ded8352633 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java index 636a22621555f..09e30e9f4de1b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java index f2e4a6c9baf27..8a5063a1a959d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java index 842ebd1b13c93..018ec4e0cb461 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java index 8b05f3004730c..b98272da45e3a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java index f1fa000a95218..253c10d84f5c1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java index 8dd41d2108847..5a7b0590dd9df 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java index dc71b5b6a6078..4e712cbbf3799 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java index deebe8d8006d3..700b618da5aff 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java index 53bf8415c9404..6343fc6c7f30c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java index b16e090df2fac..e2fb55f6cc80e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java index 79302da51c92e..22afa5fe65dd1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java index 968ab5fb0b210..a46769daf0e74 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java index 01014fd8274af..f30bb3a556437 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java index 7e8d306319172..778bb542eb398 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java index a07b6efdfdc41..2de5a936fc87f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java index 0d543e88cc5c5..4966643f120cb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java index bc37cab39b0de..7aebe04ee7559 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java index 981b07639b9ee..6a07ff82f3e0a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java index 5ef98ab309169..5c4c80a2d8136 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java index 015b3b0a943d7..85aacc78447b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java index 6181e1828d034..cca8ce4c2a76d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java index 81fbe8cc57987..72632b346125b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java index c7466b865bf34..4b9d93c90fdac 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java index 38d6011a7dfce..1d850bad7fecb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java index 348b0cf0a56d8..4ad9eea3c9f88 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java index 47b5b6db90d7b..2a476f032b9e4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java index f9c26be1911b5..eb04d1f8c9342 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java index 6342c2e323274..055d34540b733 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java index 923d329908fc8..579e74eab63d1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java index e4b90c377468e..152a0adb1bd74 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java index e06b4a8371a94..9684bc1ffa751 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java index f7c6332aaa43b..c929299013643 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java index ae44ef50cde7c..2ebee541619a1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java index f9b2565511150..1cbebc73edede 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java index 67b3738817a6f..c5717bbfc8d4e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java index 2f6c6fac985a1..46feaff451811 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java index 208055a5fdd8e..fe46cb792a521 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java index 843c985a5c9e3..93aaf155a2362 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java index 4dfbfe91e0dd6..cc6b6d958abb4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java index 7b5f4e6213a7c..1370b7bc13e04 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java index 37ed781aae2a8..2737fae2f7981 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java index a070a32e99bcf..1d535e79bbe36 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java index 34a7f14a607ee..cc96a7a8f2019 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java index 539f8c66e65d8..ebee354cb51cb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java index 3eb9b76529637..ac9df09713dba 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java index c67cce57c07b8..e469d6ea037c6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java index 9ba213b2c81e3..4614ac518c7fa 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java index a8c55c3f46d19..80c7d4d79e9f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java index 27acec96ef496..0f43ae164b352 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java index 56e084c33b698..2c57ddb62cc08 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java index cf06de0228586..e6ad99c76d1f8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java index bfd282359c1b9..dd5f607b1f57a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java index fa3e2a2f5023f..1db1f410791fe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java index d228873c4f1f9..e5f1858081299 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java index bcc725d066e80..a9cf5d2a04233 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java index 326822fa88a64..4221ae73c0749 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceWindow.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceWindow.java index 6b0e45fa4f993..1bc06b51cfa25 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceWindow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/RightJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/RightJoin.java index 0af278eb34a70..3543f82d42e72 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/RightJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/RightJoin.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SingleInputOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SingleInputOperator.java index c0672f3646585..81fa303db032e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SingleInputOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SingleInputOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAware.java index 38b8444d12d7b..db7e5b6c736b6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAware.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseOperator.java index e917e08d62fd3..d075750c411d2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java index 0baade769dd26..f8536309b8971 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateSupport.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateSupport.java index dc0af85ba82b5..39f35d5cd1ef3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateSupport.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/StateSupport.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SumByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SumByKey.java index 2fb34941b2af3..ef2cc247b146f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SumByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/SumByKey.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/TopPerKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/TopPerKey.java index e8ada51ec9dce..37421d6bd7a96 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/TopPerKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/TopPerKey.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Union.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Union.java index 8829dbb1625ca..ff19407e64e5d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Union.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Union.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java index 301619adc3ca9..253aabb87e246 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowWiseOperator.java index ae321bf7f1c91..d3b475beb5098 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowWiseOperator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowingRequiredException.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowingRequiredException.java index 087182ff6f1f7..6f7532de72f55 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowingRequiredException.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowingRequiredException.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/ComputationHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/ComputationHint.java index eb1140fcb7268..1747994c63bca 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/ComputationHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/ComputationHint.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/OutputHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/OutputHint.java index 6ccaaf0f9cc88..e7869bae991b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/OutputHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/OutputHint.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/SizeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/SizeHint.java index c128e34dae218..9cafa41e145c1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/SizeHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/SizeHint.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorage.java index 7604f412ad6f1..9fe3797203e96 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorage.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorageDescriptor.java index 9dc45a671887f..b9577862732fe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorageDescriptor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java index 28dac4625743c..51a0f8dcb7bcc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/State.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/State.java index f7478f62edf4e..6038d7f98252c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/State.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/State.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateContext.java index bbbfbd87dfd7f..7fc9c47f56b2e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateFactory.java index 665f5da498398..4eb45c04ff5f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateFactory.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateMerger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateMerger.java index ce42561a41dbb..7b1c34d27e854 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateMerger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateMerger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java index a4e082db77084..86782e2e9b821 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java index 6a6c873fa424b..7ece4c26acd45 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java index 9a5b97a33a08d..081040c520673 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java index 4aae519948d40..37f8ecff99f8a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java index c0c797264e150..b4cc105d5f77c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java index 1dee73071ebb3..03a2b0c235b04 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java index 62ef3ac2321fb..77819792d0621 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java index e206f66a2612f..157d98fe8e06b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java index b22f42f607431..e615283c308c0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java index 607f0bada3fc6..0f13f502d906a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java index d819d346fb23d..01338a19d5653 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java index 50d70ca374e94..0e77739a009ee 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java index fc9a525633355..434aa26f02dbe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java index 4c5d45353e784..6badb6a3c844d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java index b1174044a039c..992b18248e146 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java index 5247ae580fe97..2bbb5190b6f56 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java index c52b1de3158d1..3cb6afb0a16aa 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java index e10fcb2f27cb0..72b7c04f714c5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java index 965380ba4768c..36ffc608de93b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java index 9101407e93f17..6b013f8eece8a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java index f5d6ec8cb7bac..543db17204c73 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java index 772850d0849d8..812cfd415ba12 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java index 66299f82c2930..533a524882359 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java index ae43837992355..028274be91f33 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java index 2d4e8ff13d973..37923ee55354c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java index 9c47d4bab4277..465f778a3f2bc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java index 046c71a1775da..749f4518a9b8e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java index 290844e39454e..032eab9161dbe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java index ce45a9d2c40a1..61cb5506bc603 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java index 7c54df3d69f57..26bb969cf2e95 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java index 0cf38f25edefe..8deb1305c3027 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java index f40c6cb5aa395..248756c96ac5e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java index eeffaf768d035..8067210fb290d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java index 4530bfcc94469..61add66a7a899 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java index aeccdaa6ddda3..1a5ec7c2a40db 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java index d198dde85465d..e3a33a10b53c9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java index 65976c551e642..f48921599c91d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java index 75e71ed50a3d8..6ab078ba39b14 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java index 395347aaa5c28..a66b81825f34c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java index 7e823f66b9a2d..3bd36d8e980cf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java index 2f7f73f7bb37a..b2c6a91fb7b6c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java index eef302f379ecd..b68b3ab44d8e5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java index f3c8a1793a0da..c273aec24d488 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java index 32352361bc062..312d138295e2c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java index 4466ef2f54953..e7b9e01ff92f3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java index 40ae1432028ab..4c73563ba6d74 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java index 2d5860d4ab4ff..b568cadb4600b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java index e34d44fa7c4dc..7129c7d7e54cf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java index 142ad4f9dccf6..e1144dbd16d69 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java index d027173036569..12e5dfc903b71 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java index b00977b3de821..1d4e3999f0c8c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java index 544f1080fc71e..339a97fea0b78 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java index 4197c7f1bfc36..61818390547a3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java index 9f9db034102c0..8a24c1d072664 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java index 08bb5446ca884..a6193dd195137 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java index 036c561f1b0bf..8e7c818caacd7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java index 2261961f818b7..1d7d1ea1728d7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java index b363c3bd9692f..3463e38daaad6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java index 065d87effc479..591a2d827945f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java index 60789554a077e..cf27674bedb27 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java index 4988d59bd8db1..5c6438ab87d06 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java index 56c3c4d3e7660..8c61a67af7355 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java index f53b33c7b772c..71f5a0213b10f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java index e2af78f60b8e2..675ea4d0295c5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java index e0238b6100db5..626ff749bd977 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java index b9ad3ab0033f5..87eb034474ddf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java index c49c745e7b44c..8012205cccc3b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java index d88bde94319d7..0f483067644ea 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java index 3ec66dd800775..86146862a63a6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java index 77daa9936220a..5d0af03131a3c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java index f646cedc3865c..acb2a19b7e9b6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java index b2af41e09aac5..e5542525afe6e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java index 0b9e0586c93df..090bb7303d322 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java index ee3df13f5f894..3faf9e63d3f0a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java index 1817825eb70e1..a302ba77c3c63 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java index 377ebdd9436bb..169cfe4adf8f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java index e1383ccddaeba..8d47ae173f3f5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java index d1faab5e8652c..d0be6194e0993 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java index 33d48dfbc0ef4..eae07bbdcd4f7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java index 3453b1682b158..f389b52c53867 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java index e2aec8e84c243..abb3765a2dbb4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java index 8395218dbab25..135c92effaa52 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java index 9bc3762a912da..eb6e8383bf738 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java index 031e1888b50bb..673c2c035bcf7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java index 4035261da4e1f..f3bcb4ff495db 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java index 6365541ffb4d6..ecfefc9d24ecc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/resources/logback-test.xml b/sdks/java/extensions/euphoria/euphoria-core/src/test/resources/logback-test.xml deleted file mode 100644 index 8051b276b041c..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/resources/logback-test.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - - - - UTF-8 - %d [%thread] %level %logger{36} - %msg%n - true - - - - - - - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-examples/build.gradle b/sdks/java/extensions/euphoria/euphoria-examples/build.gradle deleted file mode 100644 index 556fc529a631b..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/build.gradle +++ /dev/null @@ -1,32 +0,0 @@ -apply plugin: 'com.github.johnrengelman.shadow' - -dependencies { - compile project(':beam-sdks-java-extensions-euphoria-core') - compile project(':euphoria-hadoop') - compile project(':euphoria-flink') - compile project(':euphoria-spark') - compile project(':beam-sdks-java-extensions-euphoria-local') - compile project(':euphoria-hbase') - compile project(':euphoria-kafka') - testCompile project(':beam-sdks-java-extensions-euphoria-testing') - - compileOnly "org.apache.flink:flink-statebackend-rocksdb_${scalaVersion}:${flinkVersion}" - compileOnly "org.apache.flink:flink-streaming-java_${scalaVersion}:${flinkVersion}" - - compile "org.apache.hbase:hbase-client:${hbaseVersion}" - compile "org.apache.hbase:hbase-server:${hbaseVersion}" - - runtime "org.apache.flink:flink-hadoop-compatibility_${scalaVersion}:${flinkVersion}" -} - -shadowJar { - relocate 'io.netty', 'cz.seznam.euphoria.example.io.netty' -} - -configurations { - runtime.exclude group: "org.apache.hadoop" - runtime.exclude group: "org.apache.spark" - runtime.exclude group: "org.apache.flink" - runtime.exclude group: "org.scala-lang" - runtime.exclude group: "org.mortbay.jetty" -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/pom.xml b/sdks/java/extensions/euphoria/euphoria-examples/pom.xml deleted file mode 100644 index c348315215011..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/pom.xml +++ /dev/null @@ -1,140 +0,0 @@ - - - - - euphoria-parent - cz.seznam.euphoria - 0.9-SNAPSHOT - - 4.0.0 - - euphoria-examples - jar - - ${project.groupId}:${project.artifactId} - Example programs demonstrating the Euphoria API. - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.4.3 - - - package - - shade - - - ${project.artifactId} - assembly - false - - - - - - - - - - - - - - - - - - ${project.groupId} - euphoria-core - - - ${project.groupId} - euphoria-hadoop - - - ${project.groupId} - euphoria-flink - - - ${project.groupId} - euphoria-spark - - - ${project.groupId} - euphoria-hbase - - - ${project.groupId} - euphoria-local - - - ${project.groupId} - euphoria-kafka - - - ${project.groupId} - euphoria-testing - - - - org.apache.spark - spark-core_${scala.version} - 2.0.1 - provided - - - org.apache.flink - flink-statebackend-rocksdb_${scala.version} - ${flink.version} - provided - - - org.apache.flink - flink-java - ${flink.version} - provided - - - org.apache.flink - flink-clients_${scala.version} - ${flink.version} - provided - - - org.apache.hbase - hbase-server - 1.3.1 - - - - junit - junit - test - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Executors.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Executors.java deleted file mode 100644 index 77a277c1d7c0d..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Executors.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples; - -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.executor.local.LocalExecutor; -import cz.seznam.euphoria.flink.FlinkExecutor; -import cz.seznam.euphoria.flink.TestFlinkExecutor; -import cz.seznam.euphoria.spark.SparkExecutor; -import java.io.IOException; -import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; - -/** A collection of helpers for easy allocation/creation of a specific executor. */ -public class Executors { - - /** - * Creates an executor by name or fails if the specified name is not recognized. Supported names - * are: - * - * - * - * @param executorName the name of the executor to create - * @param classes classes used by the flow to be registered for serialization - * @return a newly created executor - * @throws IllegalArgumentException if the specified name is unknown - * @throws IOException if setting up the executor fails for some reason - */ - public static Executor createExecutor(String executorName, Class... classes) - throws IOException { - - // ~ be sure to go through factories to leverage java lazy class loading; - // this avoids for example loading spark dependencies in a flink environment - final Factory f; - switch (executorName) { - case "local": - f = new LocalFactory(); - break; - case "flink-test": - f = new FlinkFactory(true); - break; - case "flink": - f = new FlinkFactory(false); - break; - case "spark-test": - f = new SparkFactory(true); - break; - case "spark": - f = new SparkFactory(false); - break; - default: - throw new IllegalArgumentException("Executor not supported: " + executorName); - } - return f.create(classes); - } - - private interface Factory { - Executor create(Class... classes) throws IOException; - } - - private static class LocalFactory implements Factory { - - @Override - public Executor create(Class... classes) throws IOException { - return new LocalExecutor(); - } - } - - private static class SparkFactory implements Factory { - - private final boolean test; - - SparkFactory(boolean test) { - this.test = test; - } - - @Override - public Executor create(Class... classes) { - final SparkExecutor.Builder builder = - SparkExecutor.newBuilder("euphoria-example").registerKryoClasses(classes); - if (test) { - return builder.local().build(); - } else { - return builder.build(); - } - } - } - - private static class FlinkFactory implements Factory { - - private final boolean test; - - FlinkFactory(boolean test) { - this.test = test; - } - - @Override - public Executor create(Class... classes) throws IOException { - if (test) { - return new TestFlinkExecutor(); - } else { - return new FlinkExecutor() - .setStateBackend(new RocksDBStateBackend("hdfs:///tmp/flink/checkpoints")) - .registerClasses(classes); - } - } - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HBaseStreamer.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HBaseStreamer.java deleted file mode 100644 index b636bf0aa0558..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HBaseStreamer.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples; - -import com.google.common.base.Preconditions; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.hadoop.SerializableWritable; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.mapreduce.KeyValueSerialization.KeyValueSerializer; -import org.apache.hadoop.io.serializer.Serializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Read and persist HBase table. */ -public class HBaseStreamer implements Serializable { - - private static final Logger LOG = LoggerFactory.getLogger(HBaseStreamer.class); - private final URI input; - private final URI output; - private final SerializableWritable conf; - private final AtomicReference> serializer = new AtomicReference<>(); - private final transient Executor executor; - private HBaseStreamer(URI input, URI output, Executor executor) { - this.input = input; - this.output = output; - this.executor = executor; - this.conf = new SerializableWritable<>(HBaseConfiguration.create()); - } - - public static void main(String[] args) throws URISyntaxException, IOException { - Preconditions.checkArgument( - args.length == 3, "Please specify "); - - URI input = new URI(args[0]); - URI output = new URI(args[1]); - Executor executor = Executors.createExecutor(args[2]); - HBaseStreamer app = new HBaseStreamer(input, output, executor); - app.run(); - } - - private void run() { - - Flow flow = Flow.create(); - Dataset> ds = - flow.createInput(Utils.getHBaseSource(input, conf.get())); - - FlatMap.of(ds) - .using( - (Pair p, Collector c) -> { - writeCellsAsBytes(p.getSecond(), c); - }) - .output() - .persist(Utils.getSink(output, conf.get())); - LOG.info("Starting flow reading from {} and persisting to {}", input, output); - executor.submit(flow).join(); - } - - private void writeCellsAsBytes(Result res, Collector c) { - - if (serializer.get() == null) { - serializer.set(new KeyValueSerializer()); - } - final Serializer s = serializer.get(); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - for (Cell cell : res.listCells()) { - try { - s.open(baos); - s.serialize((KeyValue) cell); - s.close(); - c.collect(baos.toByteArray()); - baos.reset(); - } catch (IOException ex) { - throw new RuntimeException(ex); - } - } - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HFileLoader.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HFileLoader.java deleted file mode 100644 index e412f9b3a974d..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/HFileLoader.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples; - -import static cz.seznam.euphoria.examples.Utils.getHBaseSource; -import static cz.seznam.euphoria.examples.Utils.getPath; -import static cz.seznam.euphoria.examples.Utils.getZnodeParent; -import static cz.seznam.euphoria.examples.Utils.toCell; - -import com.google.common.base.Preconditions; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.Filter; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.util.Settings; -import cz.seznam.euphoria.hadoop.input.SequenceFileSource; -import cz.seznam.euphoria.hbase.HBaseSource; -import cz.seznam.euphoria.hbase.HFileSink; -import cz.seznam.euphoria.hbase.util.ResultUtil; -import cz.seznam.euphoria.kafka.KafkaSource; -import cz.seznam.euphoria.spark.SparkExecutor; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.time.Duration; -import java.util.stream.Stream; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; - -/** Load data to HBase from various sources and in various bulks. */ -public class HFileLoader { - - private final Configuration conf = HBaseConfiguration.create(); - private final String table; - private final String outputQuorum; - private final String tmpDir; - private final URI input; - private final URI output; - private final Executor executor; - private final Flow flow; - HFileLoader(URI input, URI output, String tmpDir, Executor executor) { - this.table = getPath(output); - this.outputQuorum = output.getAuthority(); - this.tmpDir = tmpDir; - this.input = input; - this.output = output; - this.executor = executor; - this.flow = Flow.create(); - } - - public static void main(String[] args) throws URISyntaxException, IOException { - Preconditions.checkArgument( - args.length >= 3, "Please specify []"); - - URI input = new URI(args[0]); - URI output = new URI(args[1]); - Executor executor = Executors.createExecutor(args[2], KeyValue.class); - String tmp = "/tmp/hfileloader"; - if (args.length > 3) { - tmp = args[3]; - } - HFileLoader app = new HFileLoader(input, output, tmp, executor); - app.run(); - } - - @SuppressWarnings("unchecked") - private void run() { - final Dataset ds; - flow.getSettings().setInt("euphoria.flink.batch.list-storage.max-memory-elements", 100); - switch (input.getScheme()) { - case "kafka": - { - Settings settings = new Settings(); - settings.setInt(KafkaSource.CFG_RESET_OFFSET_TIMESTAMP_MILLIS, 0); - Dataset> raw = - flow.createInput(new KafkaSource(input.getAuthority(), getPath(input), settings)); - ds = MapElements.of(raw).using(p -> toCell(p.getSecond())).output(); - break; - } - case "hdfs": - case "file": - { - Dataset> raw = - flow.createInput( - new SequenceFileSource<>( - ImmutableBytesWritable.class, (Class) KeyValue.class, input.toString())); - ds = MapElements.of(raw).using(Pair::getSecond).output(); - break; - } - case "hbase": - { - HBaseSource source = getHBaseSource(input, conf); - Dataset> raw = flow.createInput(source); - - Dataset tmp = FlatMap.of(raw).using(ResultUtil.toCells()).output(); - - // this is workaround of https://issues.apache.org/jira/browse/SPARK-5928 - if (executor instanceof SparkExecutor) { - tmp = - ReduceByKey.of(tmp) - .keyBy(Object::hashCode) - .reduceBy((Stream s, Collector ctx) -> s.forEach(ctx::collect)) - .outputValues(); - } - - ds = Filter.of(tmp).by(c -> c.getValueLength() < 1024 * 1024).output(); - - break; - } - - default: - throw new IllegalArgumentException("Don't know how to load " + input); - } - - ds.persist( - HFileSink.newBuilder() - .withConfiguration(conf) - .withZookeeperQuorum(outputQuorum) - .withZnodeParent(getZnodeParent(output)) - .withTable(table) - .withOutputPath(new Path(tmpDir)) - .applyIf( - ds.isBounded(), - b -> b.windowBy(GlobalWindowing.get(), w -> ""), - b -> - b.windowBy( - Time.of(Duration.ofMinutes(5)), w -> String.valueOf(w.getStartMillis()))) - .build()); - - executor.submit(flow).join(); - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Utils.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Utils.java deleted file mode 100644 index 5c1ce3d198843..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/Utils.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples; - -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSinks; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.util.Settings; -import cz.seznam.euphoria.hadoop.output.SequenceFileSink; -import cz.seznam.euphoria.hbase.HBaseSource; -import cz.seznam.euphoria.kafka.KafkaSink; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.net.URLDecoder; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.mapreduce.KeyValueSerialization.KeyValueDeserializer; - -/** Various utilities for examples. */ -class Utils { - - static AtomicReference cache = new AtomicReference<>(); - - private Utils() {} - - static String getPath(URI uri) { - String path = uri.getPath(); - Preconditions.checkArgument(path.length() > 1, "Path must not be empty."); - return path.substring(1); - } - - static Map> splitQuery(String query) { - if (Strings.isNullOrEmpty(query)) { - return Collections.emptyMap(); - } - return Arrays.stream(query.split("&")) - .map(Utils::splitQueryParameter) - .collect( - Collectors.groupingBy( - Pair::getFirst, - HashMap::new, - Collectors.mapping(Pair::getSecond, Collectors.toList()))); - } - - static Pair splitQueryParameter(String it) { - final int idx = it.indexOf("="); - final String key = idx > 0 ? it.substring(0, idx) : it; - final String value = idx > 0 && it.length() > idx + 1 ? it.substring(idx + 1) : null; - try { - return Pair.of( - URLDecoder.decode(key, "UTF-8"), - value == null ? null : URLDecoder.decode(value, "UTF-8")); - } catch (UnsupportedEncodingException ex) { - throw new RuntimeException(ex); - } - } - - static String getZnodeParent(URI uri) { - Map> query = splitQuery(uri.getQuery()); - return Optional.ofNullable(query.get("znode")).map(l -> l.get(0)).orElse(null); - } - - static Cell toCell(byte[] input) { - try { - if (cache.get() == null) { - cache.set(new KeyValueDeserializer()); - } - KeyValueDeserializer d = cache.get(); - ByteArrayInputStream bais = new ByteArrayInputStream(input); - d.open(bais); - KeyValue ret = d.deserialize(new KeyValue()); - d.close(); - return ret; - } catch (IOException ex) { - throw new RuntimeException(ex); - } - } - - static HBaseSource getHBaseSource(URI input, Configuration conf) { - - HBaseSource.Builder builder = - HBaseSource.newBuilder() - .withConfiguration(conf) - .withZookeeperQuorum(input.getAuthority()) - .withZnodeParent(getZnodeParent(input)); - - String path = getPath(input); - if (!path.isEmpty()) { - String[] split = path.split("/", 2); - if (split.length == 2) { - builder.withTable(split[0]); - String[] parts = split[1].split(","); - for (String part : parts) { - if (!part.isEmpty()) { - String[] cf = part.split(":"); - if (cf.length == 1) { - builder.addFamily(cf[0]); - } else { - builder.addColumn(cf[0], cf[1]); - } - } - } - return builder.build(); - } - } - - throw new IllegalArgumentException( - "Invalid input URI, expected " - + "hbase:///table/[[:],]+"); - } - - static DataSink getSink(URI output, Configuration conf) { - switch (output.getScheme()) { - case "hdfs": - case "file": - return DataSinks.mapping( - SequenceFileSink.of(ImmutableBytesWritable.class, ImmutableBytesWritable.class) - .outputPath(output.toString()) - .withConfiguration(conf) - .build(), - b -> Pair.of(new ImmutableBytesWritable(), new ImmutableBytesWritable(b))); - case "kafka": - return DataSinks.mapping( - new KafkaSink(output.getAuthority(), getPath(output), toSettings(conf)), - b -> Pair.of(new byte[0], b)); - } - throw new IllegalArgumentException("Unknown scheme in " + output); - } - - private static Settings toSettings(Configuration conf) { - Settings ret = new Settings(); - for (Map.Entry e : conf) { - ret.setString(e.getKey(), e.getValue()); - } - return ret; - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/AccessLogCount.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/AccessLogCount.java deleted file mode 100644 index 0626535998810..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/AccessLogCount.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples.wordcount; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.io.StdoutSink; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.examples.Executors; -import cz.seznam.euphoria.hadoop.input.SimpleHadoopTextFileSource; -import java.text.SimpleDateFormat; -import java.time.Duration; -import java.util.Date; -import java.util.Locale; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * Simple aggregation of logs in Apache log format. Counts the number of daily hits per client. This - * is a word-count like program utilizing time windowing based on event time. - * - *

If newly coming the euphoria API, you are advised to first study the {@link SimpleWordCount} - * program. - * - *

Example usage on flink: - * - *

{@code
- * $ flink run -m yarn-cluster \
- *    -yn 1 -ys 2 -ytm 800 \
- *    -c cz.seznam.euphoria.examples.wordcount.AccessLogCount \
- *    euphoria-examples/assembly/euphoria-examples.jar \
- *    "flink" \
- *    "hdfs:///tmp/access.log"
- * }
- * - * Example usage on spark: - * - *
{@code
- * $ spark-submit --verbose --deploy-mode cluster \
- *     --master yarn \
- *     --executor-memory 1g \
- *     --num-executors 1 \
- *     --class cz.seznam.euphoria.examples.wordcount.AccessLogCount \
- *     euphoria-examples/assembly/euphoria-examples.jar \
- *     "spark" \
- *     "hdfs:///tmp/access.log"
- * }
- */ -public class AccessLogCount { - - public static void main(String[] args) throws Exception { - if (args.length < 2) { - System.err.println("Usage: " + AccessLogCount.class + " "); - System.exit(1); - } - - final String executorName = args[0]; - final String inputPath = args[1]; - - // As with the {@code SimpleWordCount} we define a source to read data from ... - final DataSource source = new SimpleHadoopTextFileSource(inputPath); - - // ... and a sink to write the business logic's output to. In this particular - // case we use a sink that eventually writes out the data to the executors - // standard output. This is rarely useful in production environments but - // is handy in local executions. - final DataSink sink = new StdoutSink<>(); - - // We start by allocating a new flow, a container to encapsulates the - // chain of transformations. - final Flow flow = Flow.create("Access log processor"); - - // From the data source describing the actual input data location and - // physical form, we create an abstract data set to be processed in the - // context of the created flow. - // - // As in other examples, reading the actual input source is deferred - // until the flow's execution. The data itself is _not_ touched at this - // point in time yet. - final Dataset input = flow.createInput(source); - - // Build flow, that transforms our input to final dataset. - final Dataset output = buildFlow(input); - - // Persist final output to data sink. - output.persist(sink); - - // Finally, we allocate an executor and submit our flow for execution on it. - final Executor executor = Executors.createExecutor(executorName); - - executor.submit(flow).get(); - } - - static Dataset buildFlow(Dataset lines) { - - // We assume the actual input data to have a particular format; in this - // case, each element is expected to be a log line from the Apache's access - // log. We "map" the "parseLine" function over each such line to transform - // the raw log entry into a more structured object. - // - // Note: Using `MapElements` implies that for each input we generate an - // output. In the context of this program it means, that we are not able - // to naturally "skip" invalid log lines. - // - // Note: Generally, user defined functions must be thread-safe. If you - // inspect the `parseLine` function, you'll see that it allocates a new - // `SimpleDateFormat` instance for every input element since sharing such - // an instance between threads without explicit synchronization is not - // thread-safe. (In this example we have intentionally used the - // `SimpleDateFormat` to make this point. In a read-world program you - // would probably hand out to `DateTimeFormatter` which can be safely - // be re-used across threads.) - final Dataset parsed = - MapElements.named("LOG-PARSER").of(lines).using(LogParser::parseLine).output(); - - // Since our log lines represent events which happened at a particular - // point in time, we want our system to treat them as such, no matter in - // which particular order the lines happen to be in the read input files. - // - // We do so by applying a so-called event-time-extractor function. As of - // this moment, euphoria will treat the element as if it happended in the - // corresponding time since it gets to know the timestamp the event occurred. - final Dataset parsedWithEventTime = - AssignEventTime.of(parsed).using(line -> line.getDate().getTime()).output(); - - // In the previous step we derived a data set specifying points in time - // at which particular IPs accessed our web-server. Our goal is now to - // count how often a particular IP accessed the web-server, per day. This - // is, instead of deriving the count of a particular IP from the whole - // input, we want to know the number of hits per IP for every day - // distinctly (we're not interested in zero hit counts, of course.) - // - // Actually, this computation is merely a word-count problem explained - // in the already mentioned {@link SimpleWordCount}. We just count the - // number of occurrences of a particular IP. However, we also specify - // how the input is to be "windowed." - // - // Windowing splits the input into fixed sections of chunks. Such as we - // can divide a data set into chunks by a certain size, we can split - // a data set into chunks defined by time, e.g. a chunk for day one, - // another chunk for day two, etc. provided that elements of the data - // set have a notion of time. Once the input data set is logically divided - // into these "time windows", the computation takes place separately on - // each of them, and, produces a results for each window separately. - // - // Here, we specify time based windowing using the `Time.of(..)` method - // specifying the size of the windows, in particular "one day" in this - // example. The assignment of an element to a particular time window, - // will, by definition, utilize the processed elements assigned timestamp. - // This is what we did in the previous step. - // - // Note: There are a few different windowing strategies and you can - // investigate each by looking for classes implementing {@link Windowing}. - // - // Note: You might wonder why we didn't just a - // "select ip, count(*) from input group by (ip, day)". First, windowing - // as such is a separate concern to the actual computation; there is no - // need to mix them up and further complicate the actual computation. - // Being a separate concern it allows for easier exchange and - // experimentation. Second, by specifying windowing as a separate concern, - // we can make the computation work even on unbounded, i.e. endless, input - // streams. Windowing strategies generally work together with the - // executor and can define a point when a window is determined to be - // "filled" at which point the windows data can be processed, calculated, - // and the corresponding results emitted. This makes endless stream - // processing work. - final Dataset> aggregated = - ReduceByKey.named("AGGREGATE") - .of(parsedWithEventTime) - .keyBy(LogLine::getIp) - .valueBy(line -> 1L) - .combineBy(Sums.ofLongs()) - .windowBy(Time.of(Duration.ofDays(1))) - .output(); - - // At the final stage of our flow, we nicely format the previously emitted - // results before persisting them to a given data sink, e.g. external storage. - // - // The elements emitted from the previous operator specify the windowed - // results of the "IP-count". This is, for each IP we get a count of the - // number of its occurrences (within a window.) The window information - // itself - if desired - can be accessed from the `FlatMap`'s context - // parameter as demonstrated below. - return FlatMap.named("FORMAT-OUTPUT") - .of(aggregated) - .using( - ((Pair elem, Collector context) -> { - Date d = new Date(((TimeInterval) context.getWindow()).getStartMillis()); - - SimpleDateFormat sdf = new SimpleDateFormat("dd/MMM/yyyy", Locale.ENGLISH); - context.collect(sdf.format(d) + "\t" + elem.getFirst() + "\t" + elem.getSecond()); - })) - .output(); - } - - private static class LogParser { - - private static Pattern pattern = - Pattern.compile("^([[0-9a-zA-z-].]+) (\\S+) (\\S+) \\[([\\w:/]+\\s[+\\-]\\d{4})\\].*"); - - static LogLine parseLine(String line) { - - Matcher matcher = pattern.matcher(line); - if (matcher.matches()) { - try { - // SDF is not thread-safe, so we need to allocate one here. Ideally, - // we'd use `DateTimeFormatter` and re-use it across input elements. - // see the corresponding note at the operator utilizing `parseLine`. - SimpleDateFormat sdf = new SimpleDateFormat("dd/MMM/yyyy:HH:mm:ss Z", Locale.ENGLISH); - - String ip = matcher.group(1); - Date date = sdf.parse(matcher.group(4)); - - return new LogLine(ip, date); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - throw new IllegalStateException("Invalid log format: " + line); - } - } - - private static class LogLine { - - private final String ip; - private final Date date; - - LogLine(String ip, Date date) { - this.ip = ip; - this.date = date; - } - - String getIp() { - return ip; - } - - Date getDate() { - return date; - } - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCount.java b/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCount.java deleted file mode 100644 index 1e195e1b35813..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCount.java +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples.wordcount; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.examples.Executors; -import cz.seznam.euphoria.hadoop.input.SimpleHadoopTextFileSource; -import cz.seznam.euphoria.hadoop.output.SimpleHadoopTextFileSink; -import java.io.IOException; -import java.util.concurrent.ExecutionException; -import java.util.regex.Pattern; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Demonstrates a very simple word-count supporting batched input without windowing. - * - *

Example usage on flink: - * - *

{@code
- * $ flink run -m yarn-cluster \
- *    -yn 1 -ys 2 -ytm 800 \
- *    -c cz.seznam.euphoria.examples.wordcount.SimpleWordCount \
- *    euphoria-examples/assembly/euphoria-examples.jar \
- *    "flink" \
- *    "hdfs:///tmp/swc-input" \
- *    "hdfs:///tmp/swc-output" \
- *    "2"
- * }
- * - * Example usage on spark: - * - *
{@code
- * $ spark-submit --verbose --deploy-mode cluster \
- *     --master yarn \
- *     --executor-memory 1g \
- *     --num-executors 1 \
- *     --class cz.seznam.euphoria.examples.wordcount.SimpleWordCount \
- *     euphoria-examples/assembly/euphoria-examples.jar \
- *     "spark" \
- *     "hdfs:///tmp/swc-input" \
- *     "hdfs:///tmp/swc-output" \
- *     "1"
- * }
- */ -public class SimpleWordCount { - - private static final Logger LOG = LoggerFactory.getLogger(SimpleWordCount.class); - - private static final Pattern SPLIT_RE = Pattern.compile("\\s+"); - - public static void main(String[] args) { - if (args.length < 3) { - System.err.println( - "Usage: " + SimpleWordCount.class + " "); - System.exit(1); - } - final String executorName = args[0]; - final String inputPath = args[1]; - final String outputPath = args[2]; - - // Define a source of data to read text lines from. We utilize an - // already predefined DataSource implementations hiding some of - // implementation details. Note that at this point in time the data - // is not read. The source files will be opened and read in a distributed - // manner only when the "WordCount" flow is submitted for execution. - final DataSource source = new SimpleHadoopTextFileSource(inputPath); - - // Define a sink where to write the program's final results to. As with - // the above defined source, no resources are opened for writing yet. - // Only when the program is submitted for execution, the sink will be - // instructed to open writers to the final, physical destination. Here, - // we utilize an already predefined DataSink which simply writes a string - // on its own line. - final DataSink sink = new SimpleHadoopTextFileSink<>(outputPath); - - // The first step in building a euphoria flow is creating a ... - // well, a `Flow` object. It is a container encapsulating a chain - // of transformations. Within a program we can have many flows. Though, - // these all will be independent. Dependencies between operation - // can be expressed only within a single flow. - // - // It is usually good practice to give each flow within a program a - // unique name to make it easier to distinguish corresponding statistics - // or otherwise displayed information from other flow which may be - // potentially part of the program. - final Flow flow = Flow.create(SimpleWordCount.class.getSimpleName()); - - // Given a data source we lift this source up into an abstract data - // set. A data set is the input and output of operators. While a source - // describes a particular source a data set is abstracting from this - // particular notion. It can be literally thought of as a "set of data" - // (without the notion of uniqueness of elements.) - // - // Note: we ask the flow to do this lifting. The new data set will - // automatically be associated with the flow. All operators processing - // this data set will also become automatically associated with the - // flow. Using the data set (or an operator) associated with a flow - // in a different flow, is considered an error and will lead to - // exceptions before the flow is even executed. - final Dataset input = flow.createInput(source); - - // Construct a flow which we'll later submit for execution. For the sake - // of readability we've moved the definition into its own method. - final Dataset output = buildFlow(input); - - // Persist final dataset to sink. - output.persist(sink); - - try { - // Allocate an executor by the specified name. - Executor executor = Executors.createExecutor(executorName); - - // Only now we submit the flow and will have the executor execute - // the business logic defined by the flow. Only, we data sources - // and sinks will be opened. - // - // As you can see the submission of flow happens in the background, - // and we could submit other flows to execute concurrently with the - // one just submitted. To await the termination of a flow, we just - // ask for the result of the `Future` object returned by `submit()`. - executor.submit(flow).get(); - } catch (InterruptedException ex) { - LOG.warn("Interrupted while waiting for the flow to finish.", ex); - } catch (IOException | ExecutionException ex) { - throw new RuntimeException(ex); - } - } - - /** - * This method defines the executor independent business logic of the program. - * - * @param lines lines of text - * @return output dataset, containing words with its count - */ - static Dataset buildFlow(Dataset lines) { - - // In the next step we want to chop up the data set of strings into a - // data set of words. Using the `FlatMap` operator we can process each - // element/string from the original data set to transform it into words. - // - // Note: Generally we are never modifying the original input data set but - // merely produce a new one. Further, the processing order of the input - // elements is generally unknown and typically happens in parallel. - // - // The `FlatMap` operator in particular is a handy choice at this point. - // It processes one input element at a time and allows user code to emit - // zero, one, or more output elements. We use it here to chop up a long - // string into individual words and emit each individually instead. - final Dataset words = - FlatMap.named("TOKENIZER") - .of(lines) - .using( - (String line, Collector c) -> - SPLIT_RE.splitAsStream(line).forEach(c::collect)) - .output(); - - // Given the "words" data set, we want to reduce it to a collection - // of word-counts, i.e. a collection which counts the number occurrences - // of every distinct word. - // - // From each input element we extract a key, which is the word itself - // here, and a value, which is the constant `1` in this example. Then, we - // reduce by the key - the operator ensures that all values for the same - // key end up being processed together. It applies our `combineBy` user - // defined function to these values. The result of this user defined - // function is then emitted to the output along with its corresponding - // key. - final Dataset> counted = - ReduceByKey.named("REDUCE") - .of(words) - .keyBy(String::toLowerCase) - .valueBy(e -> 1L) - .combineBy(Sums.ofLongs()) - .output(); - - // Lastly we merely format the output of the preceding operator and - // call `.persist()` with a data sink specifying the "persistent" - // destination of the data. A data source itself describes where to - // write the data to and how to physically lay it out. - // - // Note: a flow without any call to `.persist()` is meaningless as - // such a flow would never produces anything. Executors are free to - // reject such flows. - return MapElements.named("FORMAT") - .of(counted) - .using(p -> p.getFirst() + ":" + p.getSecond()) - .output(); - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/main/resources/log4j.properties b/sdks/java/extensions/euphoria/euphoria-examples/src/main/resources/log4j.properties deleted file mode 100644 index 0ee9259273b47..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/main/resources/log4j.properties +++ /dev/null @@ -1,7 +0,0 @@ -# Root logger option -log4j.rootLogger=INFO, stdout -# Direct log messages to stdout -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/test/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCountTest.java b/sdks/java/extensions/euphoria/euphoria-examples/src/test/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCountTest.java deleted file mode 100644 index 95fe8347b47b0..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-examples/src/test/java/cz/seznam/euphoria/examples/wordcount/SimpleWordCountTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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. - */ -package cz.seznam.euphoria.examples.wordcount; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.executor.local.LocalExecutor; -import cz.seznam.euphoria.testing.AbstractFlowTest; -import java.util.Arrays; -import java.util.List; -import org.junit.Test; - -public class SimpleWordCountTest { - - private static void execute(FlowTest test) { - test.execute(new LocalExecutor()); - } - - @Test - public void test_allLowercase() { - execute( - new FlowTest() { - - @Override - List getInput() { - return Arrays.asList( - "first second third fourth", - "first second third fourth", - "first second third fourth", - "first second third fourth"); - } - - @Override - protected List getOutput() { - return Arrays.asList("first:4", "second:4", "third:4", "fourth:4"); - } - }); - } - - @Test - public void test_firstLetterUppercase() { - execute( - new FlowTest() { - - @Override - List getInput() { - return Arrays.asList( - "First Second Third Fourth", - "First Second Third Fourth", - "First Second Third Fourth", - "First Second Third Fourth"); - } - - @Override - protected List getOutput() { - return Arrays.asList("first:4", "second:4", "third:4", "fourth:4"); - } - }); - } - - private abstract static class FlowTest extends AbstractFlowTest { - - abstract List getInput(); - - @Override - protected Dataset buildFlow(Flow flow) { - return SimpleWordCount.buildFlow(flow.createInput(ListDataSource.bounded(getInput()))); - } - } -} diff --git a/sdks/java/extensions/euphoria/euphoria-examples/src/test/resources/avro/data.avro b/sdks/java/extensions/euphoria/euphoria-examples/src/test/resources/avro/data.avro deleted file mode 100644 index 04541f9bf83eb363a24f8afc8f8abba9d7fa7758..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 272 zcmeZI%3@>^ODrqO*DFrWNXC3F?W2o*{B`8lbHc}lghwG3SggTul%|B>8#tD|;ngzEzq3nsCW V)Z!9_l*E!m1w%#_B%Tq5Y5;}WUU>ij diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/build.gradle b/sdks/java/extensions/euphoria/euphoria-fluent/build.gradle index d79044dbddbb0..5fdaa00e7abe5 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-fluent/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature() diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/pom.xml b/sdks/java/extensions/euphoria/euphoria-fluent/pom.xml deleted file mode 100644 index 0b570e2e97ca7..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-fluent/pom.xml +++ /dev/null @@ -1,62 +0,0 @@ - - - - - 4.0.0 - - cz.seznam.euphoria - euphoria-parent - 0.9-SNAPSHOT - ../pom.xml - - - euphoria-fluent - jar - - ${project.groupId}:${project.artifactId} - - A fluent API wrapper around Euphoria's client API. - - - - - cz.seznam.euphoria - euphoria-core - - - - cz.seznam.euphoria - euphoria-local - test - - - junit - junit - test - - - ch.qos.logback - logback-classic - test - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java index d1ba1c75a2820..d886570a8f33f 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java index aec8991b76b40..20015551ea7a2 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java index f779701684d11..6845fb79d3cbd 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java index bb0f758286ddd..ba57ae36a496f 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/build.gradle b/sdks/java/extensions/euphoria/euphoria-local/build.gradle index 84f0949860b32..2d37e8e12ea3b 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-local/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature() diff --git a/sdks/java/extensions/euphoria/euphoria-local/pom.xml b/sdks/java/extensions/euphoria/euphoria-local/pom.xml deleted file mode 100644 index e4fe9bbf14f09..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-local/pom.xml +++ /dev/null @@ -1,70 +0,0 @@ - - - - - euphoria-parent - cz.seznam.euphoria - 0.9-SNAPSHOT - - 4.0.0 - - euphoria-local - jar - - ${project.groupId}:${project.artifactId} - - An all-in-memory executing euphoria executor suitable - for executing flows in unit tests. - - - - - - ${project.groupId} - euphoria-core - - - - ${project.groupId} - euphoria-testing - - - - com.google.code.findbugs - jsr305 - - - - - - cz.seznam.euphoria - euphoria-operator-testkit - test - - - - junit - junit - test - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java index b3a6ea8c49519..ab3c01cee4a24 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java index ab9709e112e51..9513c7036949c 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java index 0c1b40f105c3a..fce5c4435d3ca 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java index 997d78fc5909a..3b538de69521a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java index d58d7994253c6..f01b34973cbd7 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java index cbcac349d198c..fc19609e5af05 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java index 14fd036c8940d..66035f4085d39 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java index 7d222f540e59a..ca4591701fea9 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java index fb5ad98f8c40f..719e809f112dc 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java index ea49e418fdf49..b4164e9789adc 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java index 01798f0ab566e..7c171d18b8777 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java index 97af61cf91384..a1fa47827e1f0 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java index acc05b94ab303..82cbf8de11b6d 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java index 677bf80d9585b..9fdc9c3609e0e 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java index 34e777f1c243a..325b49f3a94e2 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java index 487ba96703d35..2ebd677ebed33 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java index d278ba07de90b..1f821d323ba0f 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java index c7ef0d47cc876..23db8e95b9065 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java index 6401f14aea879..ba1c4deee62da 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java index 90f0d50f95f61..56a22cc7027aa 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java index 24594357ff989..29add65591cd4 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java index 066dce96f0043..37fe0a81d9bb1 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java index 3f55d5791dc9c..ae9c46bcd380a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java index 2938cc9fdef59..3d536c1ec632a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java index 08beeb432085f..1f969dc2e002a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java index e8192f7504da5..346ce2283370c 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java index 538faf79cb6f3..73cc9d659d911 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle b/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle index 44ebdb4241032..3b0fce2be4e20 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature() diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/pom.xml b/sdks/java/extensions/euphoria/euphoria-operator-testkit/pom.xml deleted file mode 100644 index 4056b730e1b56..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/pom.xml +++ /dev/null @@ -1,96 +0,0 @@ - - - - - 4.0.0 - - cz.seznam.euphoria - euphoria-parent - 0.9-SNAPSHOT - ../pom.xml - - - euphoria-operator-testkit - jar - - ${project.groupId}:${project.artifactId} - - Definition of tests for executors to pass to validate themselves - as properly implementing the behavior of all operators. - - - - - - maven-surefire-plugin - - 1 - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - - - - ${project.groupId} - euphoria-core - - - - com.google.code.findbugs - jsr305 - - - - org.slf4j - slf4j-api - - - - ch.qos.logback - logback-classic - - - - junit - junit - compile - - - - org.mockito - mockito-all - compile - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java index 76a8a8bce8c50..624cd05885117 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java index d81c4c1749421..2784b3a4107fa 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java index df0558525e561..970dba3d6e1b0 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java index 86d20e0a478f6..c95a21e39ab67 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java index ff6f07946b3e7..ca915e981c579 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java index 5256bb2be8e54..dc144a9718569 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java index fbf830475a9e6..97cd3addecc85 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java index e8ed22e00cd0b..c3cd381cba949 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java index 5dc87194b93ab..2fccde64c06be 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java index 41f8bfdb85b79..08d59066b7464 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java index 7408d153a6cb3..9bb6b2c2978c5 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java index d8b38f09bfe2e..b76921b738b2c 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java index 09d66960589ca..0ce17ba8604ca 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java index 484b9f4181e8f..462fc4591e84f 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java index b7e322e9d7628..c98ca14f92c67 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java index 6008de4cd2c1d..e1f2d04e20683 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java index 9c81886f50a4b..301bf695c3351 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java index 8c739f98cd4ed..1978214dfea8d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java index 3651f4ddf66ca..90701f06558f1 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java index 8c95267600b7e..edfb551d73bea 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java index e9e3cd6f3a295..7c0bbdb20de91 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java index a9b5f55fd9484..a2e52ecb3b3f9 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java index 3a8955e8ae113..fbb12ad727620 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java index 8365d980e4abc..b95e15f7a56d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java index c418abd1445e7..8ad51e1af1137 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java index b492f5f8e91b4..7b11e9ca1ccb4 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java index 61711f2191edb..3d165743f5180 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java index 2dabac6899f1a..8bbc6e860ef69 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java index 16b48ee886b67..64890c3e0915d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java index 661042990946a..9c5aceaab3fd5 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java index ecf4d0ae57bbd..664ed89ec8ac8 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-testing/build.gradle b/sdks/java/extensions/euphoria/euphoria-testing/build.gradle index 7639102f6c795..8100b2d27e996 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-testing/build.gradle @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + apply from: project(":").file("build_rules.gradle") applyJavaNature() diff --git a/sdks/java/extensions/euphoria/euphoria-testing/pom.xml b/sdks/java/extensions/euphoria/euphoria-testing/pom.xml deleted file mode 100644 index 548483f6c9e57..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-testing/pom.xml +++ /dev/null @@ -1,64 +0,0 @@ - - - - - 4.0.0 - - cz.seznam.euphoria - euphoria-parent - 0.9-SNAPSHOT - ../pom.xml - - - euphoria-testing - jar - - ${project.groupId}:${project.artifactId} - - Euphoria testing module provides functionality for running unit tests. - - - - - - cz.seznam.euphoria - euphoria-core - - - - com.google.code.findbugs - jsr305 - - - - org.slf4j - slf4j-api - - - - junit - junit - compile - - - - - diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java index 43a48d9d80575..09f77775512e4 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java index cba2bb16853cc..1c1c8c4b7c44d 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 * From 5b6421c3f3c7207da13d7df77f893bfa0cf3a63c Mon Sep 17 00:00:00 2001 From: David Moravek Date: Tue, 15 May 2018 21:19:09 +0200 Subject: [PATCH 2/3] [BEAM-3902] Move euphoria to apache namespace. --- .../euphoria/beam/BeamExecutor.java | 12 +-- .../euphoria/beam/BeamExecutorContext.java | 55 ++++++------- .../extensions}/euphoria/beam/BeamFlow.java | 27 ++++--- .../euphoria/beam/DoFnCollector.java | 25 +++--- .../euphoria/beam/FlatMapTranslator.java | 13 +-- .../euphoria/beam/FlowTranslator.java | 35 ++++---- .../euphoria/beam/InputTranslator.java | 13 +-- .../beam/LazyAccumulatorProvider.java | 13 +-- .../euphoria/beam/OperatorTranslator.java | 4 +- .../euphoria/beam/ReduceByKeyTranslator.java | 21 ++--- .../beam/ReduceStateByKeyTranslator.java | 4 +- .../euphoria/beam/SingleValueCollector.java | 15 ++-- .../extensions}/euphoria/beam/TypeUtils.java | 2 +- .../euphoria/beam/UnionTranslator.java | 4 +- .../beam/WrappedPCollectionOperator.java | 11 +-- .../euphoria/beam/coder/PairCoder.java | 15 ++-- .../euphoria/beam/coder/package-info.java | 2 +- .../euphoria/beam/io/BeamBoundedSource.java | 13 +-- .../euphoria/beam/io/BeamUnboundedSource.java | 15 ++-- .../euphoria/beam/io/BeamWriteSink.java | 11 +-- .../euphoria/beam/io/KryoCoder.java | 9 ++- .../euphoria/beam/io/package-info.java | 2 +- .../euphoria/beam/package-info.java | 2 +- .../euphoria/beam/window/BeamWindow.java | 4 +- .../euphoria/beam/window/BeamWindowFn.java | 21 ++--- .../beam/window/BeamWindowedElement.java | 7 +- .../euphoria/beam/window/package-info.java | 2 +- .../euphoria/beam/BeamFlowTest.java | 37 ++++----- .../euphoria/beam/FlatMapTest.java | 17 ++-- .../euphoria/beam/ReduceByKeyTest.java | 67 +++++++-------- .../beam/testkit/BeamExecutorProvider.java | 13 +-- .../beam/testkit/BeamOperatorsSuite.java | 12 +-- .../beam/testkit/BeamOperatorsTest.java | 2 +- .../dataset/windowing/package-info.java | 19 ----- .../core/client/flow/package-info.java | 19 ----- .../core/client/functional/package-info.java | 19 ----- .../euphoria/core/client/io/package-info.java | 19 ----- .../core/client/lib/package-info.java | 19 ----- .../client/operator/hint/package-info.java | 19 ----- .../core/client/operator/package-info.java | 19 ----- .../client/operator/state/package-info.java | 19 ----- .../core/client/triggers/package-info.java | 19 ----- .../core/client/type/package-info.java | 19 ----- .../core/client/util/package-info.java | 19 ----- .../core/executor/graph/package-info.java | 19 ----- .../core/executor/greduce/package-info.java | 19 ----- .../core/executor/io/package-info.java | 19 ----- .../euphoria/core/executor/package-info.java | 19 ----- .../core/executor/util/package-info.java | 19 ----- .../euphoria/core/time/package-info.java | 19 ----- .../euphoria/core/util/package-info.java | 19 ----- .../core/annotation/audience/Audience.java | 2 +- .../annotation/audience/package-info.java | 2 +- .../core/annotation/operator/Basic.java | 2 +- .../core/annotation/operator/Derived.java | 2 +- .../core/annotation/operator/Recommended.java | 2 +- .../annotation/operator/StateComplexity.java | 2 +- .../annotation/operator/package-info.java | 19 +++++ .../annotation/stability/Experimental.java | 2 +- .../annotation/stability/package-info.java | 19 +++++ .../core/client/accumulators/Accumulator.java | 4 +- .../accumulators/AccumulatorProvider.java | 7 +- .../core/client/accumulators/Counter.java | 4 +- .../core/client/accumulators/Histogram.java | 4 +- .../core/client/accumulators/Timer.java | 5 +- .../accumulators/VoidAccumulatorProvider.java | 11 +-- .../client/accumulators/package-info.java | 19 +++++ .../euphoria/core/client/dataset/Dataset.java | 15 ++-- .../core/client/dataset/Datasets.java | 10 +-- .../core/client/dataset/InputDataset.java | 15 ++-- .../core/client/dataset/OutputDataset.java | 15 ++-- .../core/client/dataset/package-info.java | 19 +++++ .../core/client/dataset/windowing/Count.java | 10 +-- .../dataset/windowing/GlobalWindowing.java | 11 +-- .../dataset/windowing/MergingWindowing.java | 7 +- .../client/dataset/windowing/Session.java | 21 ++--- .../core/client/dataset/windowing/Time.java | 19 ++--- .../dataset/windowing/TimeInterval.java | 4 +- .../client/dataset/windowing/TimeSliding.java | 15 ++-- .../core/client/dataset/windowing/Window.java | 5 +- .../dataset/windowing/WindowedElement.java | 4 +- .../client/dataset/windowing/Windowing.java | 7 +- .../dataset/windowing/package-info.java | 19 +++++ .../euphoria/core/client/flow/Flow.java | 27 ++++--- .../euphoria/core/client/flow/Util.java | 2 +- .../core/client/flow/package-info.java | 19 +++++ .../client/functional/BinaryFunction.java | 5 +- .../core/client/functional/BinaryFunctor.java | 7 +- .../functional/CombinableBinaryFunction.java | 4 +- .../functional/CombinableReduceFunction.java | 4 +- .../functional/CompositeUnaryFunction.java | 4 +- .../core/client/functional/Consumer.java | 5 +- .../client/functional/ExtractEventTime.java | 5 +- .../client/functional/ReduceFunction.java | 5 +- .../core/client/functional/ReduceFunctor.java | 5 +- .../functional/SameArgsBinaryFunction.java | 4 +- .../client/functional/TernaryFunction.java | 5 +- .../core/client/functional/TypeHintAware.java | 7 +- .../core/client/functional/UnaryFunction.java | 5 +- .../client/functional/UnaryFunctionEnv.java | 7 +- .../core/client/functional/UnaryFunctor.java | 7 +- .../client/functional/UnaryPredicate.java | 4 +- .../core/client/functional/VoidFunction.java | 5 +- .../core/client/functional/package-info.java | 19 +++++ .../core/client/io/BoundedDataSource.java | 5 +- .../core/client/io/BoundedReader.java | 5 +- .../core/client/io/CloseableIterator.java | 5 +- .../euphoria/core/client/io/Collector.java | 4 +- .../euphoria/core/client/io/Context.java | 4 +- .../euphoria/core/client/io/DataSink.java | 7 +- .../euphoria/core/client/io/DataSinks.java | 9 ++- .../euphoria/core/client/io/DataSource.java | 5 +- .../euphoria/core/client/io/Environment.java | 12 +-- .../core/client/io/ExternalIterable.java | 5 +- .../euphoria/core/client/io/ListDataSink.java | 11 +-- .../core/client/io/ListDataSource.java | 5 +- .../core/client/io/MultiDataSink.java | 7 +- .../euphoria/core/client/io/SpillTools.java | 7 +- .../euphoria/core/client/io/StdoutSink.java | 5 +- .../core/client/io/UnboundedDataSource.java | 5 +- .../core/client/io/UnboundedPartition.java | 5 +- .../core/client/io/UnboundedReader.java | 4 +- .../client/io/UnsplittableBoundedSource.java | 5 +- .../euphoria/core/client/io/VoidSink.java | 5 +- .../euphoria/core/client/io/Writer.java | 5 +- .../core/client/io}/package-info.java | 2 +- .../euphoria/core/client/lib/Split.java | 17 ++-- .../core/client/lib/package-info.java | 19 +++++ .../core/client/operator/AssignEventTime.java | 19 ++--- .../core/client/operator/Builders.java | 20 ++--- .../core/client/operator/CountByKey.java | 27 ++++--- .../core/client/operator/Distinct.java | 29 +++---- .../client/operator/ElementWiseOperator.java | 11 +-- .../euphoria/core/client/operator/Filter.java | 19 ++--- .../core/client/operator/FlatMap.java | 23 +++--- .../core/client/operator/FullJoin.java | 17 ++-- .../euphoria/core/client/operator/Join.java | 45 ++++++----- .../core/client/operator/LeftJoin.java | 17 ++-- .../core/client/operator/MapElements.java | 21 ++--- .../core/client/operator/Operator.java | 17 ++-- .../operator/OptionalMethodBuilder.java | 7 +- .../core/client/operator/ReduceByKey.java | 65 +++++++-------- .../client/operator/ReduceStateByKey.java | 37 ++++----- .../core/client/operator/ReduceWindow.java | 37 ++++----- .../core/client/operator/RightJoin.java | 19 ++--- .../client/operator/SingleInputOperator.java | 9 ++- .../core/client/operator/StateAware.java | 6 +- .../StateAwareWindowWiseOperator.java | 13 +-- ...ateAwareWindowWiseSingleInputOperator.java | 19 ++--- .../core/client/operator/StateSupport.java | 6 +- .../core/client/operator/SumByKey.java | 29 +++---- .../core/client/operator/TopPerKey.java | 45 ++++++----- .../euphoria/core/client/operator/Union.java | 19 ++--- .../core/client/operator/WindowAware.java | 8 +- .../client/operator/WindowWiseOperator.java | 11 +-- .../operator/WindowingRequiredException.java | 4 +- .../client/operator/hint/ComputationHint.java | 4 +- .../core/client/operator/hint/OutputHint.java | 5 +- .../core/client/operator/hint/SizeHint.java | 4 +- .../client/operator/hint/package-info.java | 19 +++++ .../core/client/operator/package-info.java | 19 +++++ .../client/operator/state/ListStorage.java | 4 +- .../operator/state/ListStorageDescriptor.java | 4 +- .../state/MergingStorageDescriptor.java | 6 +- .../core/client/operator/state/State.java | 6 +- .../client/operator/state/StateContext.java | 7 +- .../client/operator/state/StateFactory.java | 11 +-- .../client/operator/state/StateMerger.java | 7 +- .../core/client/operator/state/Storage.java | 4 +- .../operator/state/StorageDescriptor.java | 5 +- .../operator/state/StorageProvider.java | 5 +- .../client/operator/state/ValueStorage.java | 4 +- .../state/ValueStorageDescriptor.java | 6 +- .../client/operator/state/package-info.java | 19 +++++ .../triggers/AfterFirstCompositeTrigger.java | 7 +- .../core/client/triggers/CountTrigger.java | 10 +-- .../core/client/triggers/NoopTrigger.java | 6 +- .../client/triggers/PeriodicTimeTrigger.java | 10 +-- .../core/client/triggers/TimeTrigger.java | 6 +- .../core/client/triggers/Trigger.java | 13 +-- .../core/client/triggers/TriggerContext.java | 10 +-- .../core/client/triggers/package-info.java | 19 +++++ .../core/client/type/AbstractTypeAware.java | 5 +- .../client/type/TypeAwareReduceFunctor.java | 7 +- .../client/type/TypeAwareUnaryFunction.java | 6 +- .../client/type/TypeAwareUnaryFunctor.java | 6 +- .../euphoria/core/client/type/TypeHint.java | 5 +- .../core/client/type/package-info.java | 19 +++++ .../euphoria/core/client/util/Either.java | 7 +- .../euphoria/core/client/util/Fold.java | 15 ++-- .../euphoria/core/client/util/Max.java | 9 ++- .../euphoria/core/client/util/Pair.java | 5 +- .../euphoria/core/client/util/Sums.java | 7 +- .../euphoria/core/client/util/Triple.java | 5 +- .../core/client/util/package-info.java | 19 +++++ .../core/executor/AbstractExecutor.java | 9 ++- .../euphoria/core/executor/Constants.java | 4 +- .../euphoria/core/executor/Executor.java | 19 ++--- .../euphoria/core/executor/FlowUnfolder.java | 19 ++--- .../euphoria/core/executor/FlowValidator.java | 27 ++++--- .../euphoria/core/executor/VectorClock.java | 5 +- .../euphoria/core/executor/graph/DAG.java | 7 +- .../euphoria/core/executor/graph/Node.java | 7 +- .../core/executor/graph/package-info.java | 19 +++++ .../core/executor/greduce/GroupReducer.java | 59 +++++++------- .../core/executor/greduce/TimerSupport.java | 9 ++- .../core/executor/greduce/TriggerStorage.java | 21 ++--- .../core/executor/greduce/package-info.java | 19 +++++ .../executor/io/FsSpillingListStorage.java | 9 ++- .../core/executor/io/GenericSpillTools.java | 17 ++-- .../core/executor/io/SerializerFactory.java | 5 +- .../core/executor/io/SpillFileFactory.java | 5 +- .../core/executor/io/package-info.java | 19 +++++ .../euphoria/core/executor}/package-info.java | 2 +- .../executor/util/InMemExternalIterable.java | 7 +- .../core/executor/util/MultiValueContext.java | 19 ++--- .../executor/util/OperatorTranslator.java | 17 ++-- .../executor/util/SingleValueContext.java | 17 ++-- .../core/executor/util/package-info.java | 19 +++++ .../euphoria/core/time/Scheduler.java | 5 +- .../euphoria/core/time/TimeProvider.java | 5 +- .../euphoria/core/time/TimeProviderAware.java | 4 +- .../euphoria/core/time/TimeProviders.java | 5 +- .../euphoria/core/time/TimerScheduler.java | 9 ++- .../euphoria/core/time}/package-info.java | 2 +- .../euphoria/core/util/ExceptionUtils.java | 2 +- .../euphoria/core/util/IOUtils.java | 2 +- .../euphoria/core/util/InstanceUtils.java | 5 +- .../euphoria/core/util/Settings.java | 11 +-- .../euphoria/core/util}/package-info.java | 2 +- .../client/dataset/windowing/SessionTest.java | 13 +-- .../dataset/windowing/TimeSlidingTest.java | 9 ++- .../client/dataset/windowing/TimeTest.java | 9 ++- .../dataset/windowing/TimestampedElement.java | 2 +- .../dataset/windowing/WindowingTest.java | 15 ++-- .../euphoria/core/client/flow/TestFlow.java | 19 ++--- .../euphoria/core/client/io/EmptyReader.java | 2 +- .../core/client/io/ListDataSinkTest.java | 8 +- .../core/client/io/MockStreamDataSource.java | 2 +- .../core/client/io/MultiDataSinkTest.java | 11 +-- .../euphoria/core/client/lib/SplitTest.java | 23 +++--- .../core/client/operator/CountByKeyTest.java | 18 ++--- .../core/client/operator/DistinctTest.java | 15 ++-- .../core/client/operator/FilterTest.java | 10 +-- .../core/client/operator/FlatMapTest.java | 15 ++-- .../core/client/operator/HintTest.java | 35 ++++---- .../core/client/operator/JoinTest.java | 26 +++--- .../core/client/operator/MapElementsTest.java | 14 ++-- .../core/client/operator/ReduceByKeyTest.java | 23 +++--- .../client/operator/ReduceStateByKeyTest.java | 32 ++++---- .../client/operator/ReduceWindowTest.java | 25 +++--- .../core/client/operator/SumByKeyTest.java | 18 ++--- .../core/client/operator/TopPerKeyTest.java | 20 +++-- .../core/client/operator/UnionTest.java | 10 +-- .../euphoria/core/client/operator/Util.java | 11 +-- .../core/client/util/IOUtilsTest.java | 11 +-- .../core/executor/FlowUnfolderTest.java | 53 ++++++------ .../euphoria/core/executor/graph/DAGTest.java | 11 +-- .../io/FsSpillingListStorageTest.java | 17 ++-- .../executor/io/GenericSpillToolsTest.java | 17 ++-- .../executor/io/JavaSerializationFactory.java | 2 +- .../io/TmpFolderSpillFileFactory.java | 5 +- .../euphoria/core/testing/DatasetAssert.java | 5 +- .../euphoria/core/time/TimeProvidersTest.java | 9 ++- .../euphoria/core/util/SettingsTest.java | 6 +- .../extensions}/euphoria/fluent/Dataset.java | 30 +++---- .../sdk/extensions}/euphoria/fluent/Flow.java | 16 ++-- .../extensions}/euphoria/fluent/Fluent.java | 6 +- .../euphoria/fluent/package-info.java | 2 +- .../euphoria/fluent/FluentTest.java | 21 ++--- .../local/AbstractTriggerScheduler.java | 7 +- .../executor/local/AttachedWindowing.java | 13 +-- .../euphoria/executor/local/Collector.java | 2 +- .../euphoria/executor/local/Datum.java | 6 +- .../euphoria/executor/local/ExecPath.java | 6 +- .../euphoria/executor/local/ExecUnit.java | 11 +-- .../euphoria/executor/local/KeyedWindow.java | 5 +- .../executor/local/LocalExecutor.java | 69 ++++++++-------- .../executor/local/LocalSpillTools.java | 9 ++- .../executor/local/LocalStateContext.java | 8 +- .../executor/local/LocalStorageProvider.java | 13 +-- .../executor/local/NoopTriggerScheduler.java | 4 +- .../local/ProcessingTimeTriggerScheduler.java | 2 +- .../local/ReduceStateByKeyReducer.java | 63 ++++++++------- .../executor/local/TriggerScheduler.java | 4 +- .../euphoria/executor/local/Triggerable.java | 4 +- .../executor/local/WatermarkEmitStrategy.java | 3 +- .../local/WatermarkTriggerScheduler.java | 7 +- .../local/WindowedElementCollector.java | 23 +++--- .../executor/local}/package-info.java | 2 +- .../executor/local/BasicOperatorTest.java | 53 ++++++------ .../executor/local/JoinOperatorTest.java | 43 +++++----- .../executor/local/LocalExecutorTest.java | 81 +++++++++---------- .../euphoria/executor/local/Util.java | 2 +- .../executor/local/VectorClockTest.java | 8 +- .../executor/local/WindowingTest.java | 51 ++++++------ .../local/testkit/LocalExecutorProvider.java | 12 +-- .../local/testkit/LocalOperatorTest.java | 4 +- .../operator/test/AllOperatorsSuite.java | 6 +- .../operator/test/BroadcastHashJoinTest.java | 25 +++--- .../operator/test/CountByKeyTest.java | 21 ++--- .../euphoria/operator/test/DistinctTest.java | 23 +++--- .../euphoria/operator/test/FilterTest.java | 13 +-- .../euphoria/operator/test/FlatMapTest.java | 21 ++--- .../euphoria/operator/test/IntWindow.java | 4 +- .../euphoria/operator/test/JoinTest.java | 55 ++++++------- .../test/JoinWindowEnforcementTest.java | 33 ++++---- .../operator/test/MapElementsTest.java | 21 ++--- .../operator/test/ReduceByKeyTest.java | 77 +++++++++--------- .../operator/test/ReduceStateByKeyTest.java | 69 ++++++++-------- .../operator/test/ReduceWindowTest.java | 19 ++--- .../euphoria/operator/test/SinkTest.java | 27 ++++--- .../euphoria/operator/test/SumByKeyTest.java | 17 ++-- .../euphoria/operator/test/TopPerKeyTest.java | 17 ++-- .../euphoria/operator/test/UnionTest.java | 19 ++--- .../euphoria/operator/test/Util.java | 11 +-- .../euphoria/operator/test/WatermarkTest.java | 27 ++++--- .../euphoria/operator/test/WindowingTest.java | 65 +++++++-------- .../test/accumulators/LongCounter.java | 5 +- .../test/accumulators/LongHistogram.java | 5 +- .../test/accumulators/NanosecondTimer.java | 5 +- .../SingleJvmAccumulatorProvider.java | 17 ++-- .../test/accumulators/SnapshotProvider.java | 2 +- .../test/accumulators/Snapshotable.java | 2 +- .../test/accumulators/package-info.java | 2 +- .../test/junit/AbstractOperatorTest.java | 30 +++---- .../test/junit/ExecutorEnvironment.java | 4 +- .../operator/test/junit/ExecutorProvider.java | 2 +- .../test/junit/ExecutorProviderRunner.java | 25 +++--- .../operator/test/junit/Processing.java | 3 +- .../operator/test/junit/package-info.java | 2 +- .../euphoria/operator/test/package-info.java | 4 +- .../euphoria/testing/AbstractFlowTest.java | 11 +-- .../euphoria/testing/DatasetAssert.java | 5 +- .../euphoria/testing}/package-info.java | 2 +- 335 files changed, 2388 insertions(+), 2232 deletions(-) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/BeamExecutor.java (84%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/BeamExecutorContext.java (80%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/BeamFlow.java (90%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/DoFnCollector.java (76%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/FlatMapTranslator.java (88%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/FlowTranslator.java (78%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/InputTranslator.java (84%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/LazyAccumulatorProvider.java (80%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/OperatorTranslator.java (91%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/ReduceByKeyTranslator.java (90%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/ReduceStateByKeyTranslator.java (95%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/SingleValueCollector.java (79%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/TypeUtils.java (96%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/UnionTranslator.java (91%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/WrappedPCollectionOperator.java (86%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/coder/PairCoder.java (96%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/coder/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/io/BeamBoundedSource.java (92%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/io/BeamUnboundedSource.java (95%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/io/BeamWriteSink.java (92%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/io/KryoCoder.java (96%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/io/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/package-info.java (94%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/BeamWindow.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/BeamWindowFn.java (88%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/BeamWindowedElement.java (86%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/BeamFlowTest.java (87%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/FlatMapTest.java (80%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/ReduceByKeyTest.java (78%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/testkit/BeamExecutorProvider.java (81%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/testkit/BeamOperatorsSuite.java (76%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/testkit/BeamOperatorsTest.java (93%) delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/package-info.java delete mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/audience/Audience.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/audience/package-info.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/operator/Basic.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/operator/Derived.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/operator/Recommended.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/operator/StateComplexity.java (94%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/annotation/stability/Experimental.java (95%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/Accumulator.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/AccumulatorProvider.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/Counter.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/Histogram.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/Timer.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/accumulators/VoidAccumulatorProvider.java (93%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/Dataset.java (84%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/Datasets.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/InputDataset.java (80%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/OutputDataset.java (81%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/Count.java (84%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/GlobalWindowing.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/MergingWindowing.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/Session.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/Time.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/TimeInterval.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/TimeSliding.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/Window.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/WindowedElement.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/Windowing.java (87%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/flow/Flow.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/flow/Util.java (94%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/BinaryFunction.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/BinaryFunctor.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/CombinableBinaryFunction.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/CombinableReduceFunction.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/CompositeUnaryFunction.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/Consumer.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/ExtractEventTime.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/ReduceFunction.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/ReduceFunctor.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/SameArgsBinaryFunction.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/TernaryFunction.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/TypeHintAware.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/UnaryFunction.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/UnaryFunctionEnv.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/UnaryFunctor.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/UnaryPredicate.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/functional/VoidFunction.java (86%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/BoundedDataSource.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/BoundedReader.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/CloseableIterator.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/Collector.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/Context.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/DataSink.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/DataSinks.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/DataSource.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/Environment.java (79%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/ExternalIterable.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/ListDataSink.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/ListDataSource.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/MultiDataSink.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/SpillTools.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/StdoutSink.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/UnboundedDataSource.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/UnboundedPartition.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/UnboundedReader.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/UnsplittableBoundedSource.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/VoidSink.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/Writer.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam/euphoria/core/client/dataset => org/apache/beam/sdk/extensions/euphoria/core/client/io}/package-info.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/lib/Split.java (87%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/AssignEventTime.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Builders.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/CountByKey.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Distinct.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ElementWiseOperator.java (78%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Filter.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/FlatMap.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/FullJoin.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Join.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/LeftJoin.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/MapElements.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Operator.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/OptionalMethodBuilder.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceByKey.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceStateByKey.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceWindow.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/RightJoin.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/SingleInputOperator.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/StateAware.java (84%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/StateAwareWindowWiseOperator.java (77%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java (76%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/StateSupport.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/SumByKey.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/TopPerKey.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Union.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/WindowAware.java (76%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/WindowWiseOperator.java (78%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/WindowingRequiredException.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/hint/ComputationHint.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/hint/OutputHint.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/hint/SizeHint.java (87%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/ListStorage.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/ListStorageDescriptor.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/MergingStorageDescriptor.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/State.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/StateContext.java (84%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/StateFactory.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/StateMerger.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/Storage.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/StorageDescriptor.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/StorageProvider.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/ValueStorage.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/state/ValueStorageDescriptor.java (93%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/CountTrigger.java (81%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/NoopTrigger.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/PeriodicTimeTrigger.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/TimeTrigger.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/Trigger.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/triggers/TriggerContext.java (82%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/type/AbstractTypeAware.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/type/TypeAwareReduceFunctor.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/type/TypeAwareUnaryFunction.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/type/TypeAwareUnaryFunctor.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/type/TypeHint.java (94%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Either.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Fold.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Max.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Pair.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Sums.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/Triple.java (92%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/AbstractExecutor.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/Constants.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/Executor.java (79%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/FlowUnfolder.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/FlowValidator.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/VectorClock.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/graph/DAG.java (97%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/graph/Node.java (95%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/greduce/GroupReducer.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/greduce/TimerSupport.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/greduce/TriggerStorage.java (82%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/FsSpillingListStorage.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/GenericSpillTools.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/SerializerFactory.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/SpillFileFactory.java (92%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam/euphoria/core/client/accumulators => org/apache/beam/sdk/extensions/euphoria/core/executor}/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/util/InMemExternalIterable.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/util/MultiValueContext.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/util/OperatorTranslator.java (73%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/util/SingleValueContext.java (82%) create mode 100644 sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/package-info.java rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/Scheduler.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/TimeProvider.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/TimeProviderAware.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/TimeProviders.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/TimerScheduler.java (92%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam/euphoria/core/annotation/operator => org/apache/beam/sdk/extensions/euphoria/core/time}/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/util/ExceptionUtils.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/util/IOUtils.java (97%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/util/InstanceUtils.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/util/Settings.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/main/java/{cz/seznam/euphoria/core/annotation/stability => org/apache/beam/sdk/extensions/euphoria/core/util}/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/SessionTest.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/TimeSlidingTest.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/TimeTest.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/TimestampedElement.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/dataset/windowing/WindowingTest.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/flow/TestFlow.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/EmptyReader.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/ListDataSinkTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/MockStreamDataSource.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/io/MultiDataSinkTest.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/lib/SplitTest.java (83%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/CountByKeyTest.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/DistinctTest.java (88%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/FilterTest.java (89%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/FlatMapTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/HintTest.java (78%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/JoinTest.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/MapElementsTest.java (90%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceByKeyTest.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceStateByKeyTest.java (84%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/ReduceWindowTest.java (86%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/SumByKeyTest.java (85%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/TopPerKeyTest.java (87%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/UnionTest.java (94%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/operator/Util.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/client/util/IOUtilsTest.java (93%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/FlowUnfolderTest.java (82%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/graph/DAGTest.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/FsSpillingListStorageTest.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/GenericSpillToolsTest.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/JavaSerializationFactory.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/executor/io/TmpFolderSpillFileFactory.java (95%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/testing/DatasetAssert.java (96%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/time/TimeProvidersTest.java (98%) rename sdks/java/extensions/euphoria/euphoria-core/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/core/util/SettingsTest.java (96%) rename sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/fluent/Dataset.java (62%) rename sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/fluent/Flow.java (67%) rename sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/fluent/Fluent.java (82%) rename sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/fluent/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/fluent/FluentTest.java (80%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/AbstractTriggerScheduler.java (98%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/AttachedWindowing.java (74%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/Collector.java (93%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/Datum.java (93%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/ExecPath.java (84%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/ExecUnit.java (87%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/KeyedWindow.java (91%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/LocalExecutor.java (92%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/LocalSpillTools.java (84%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/LocalStateContext.java (79%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/LocalStorageProvider.java (80%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/NoopTriggerScheduler.java (90%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/ProcessingTimeTriggerScheduler.java (94%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/ReduceStateByKeyReducer.java (92%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/TriggerScheduler.java (94%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/Triggerable.java (91%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/WatermarkEmitStrategy.java (96%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/WatermarkTriggerScheduler.java (95%) rename sdks/java/extensions/euphoria/euphoria-local/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/WindowedElementCollector.java (73%) rename sdks/java/extensions/euphoria/{euphoria-testing/src/main/java/cz/seznam/euphoria/testing => euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local}/package-info.java (92%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/BasicOperatorTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/JoinOperatorTest.java (83%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/LocalExecutorTest.java (86%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/Util.java (95%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/VectorClockTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/WindowingTest.java (88%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/testkit/LocalExecutorProvider.java (74%) rename sdks/java/extensions/euphoria/euphoria-local/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/executor/local/testkit/LocalOperatorTest.java (86%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/AllOperatorsSuite.java (85%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/BroadcastHashJoinTest.java (87%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/CountByKeyTest.java (85%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/DistinctTest.java (84%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/FilterTest.java (80%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/FlatMapTest.java (83%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/IntWindow.java (90%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/JoinTest.java (90%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/JoinWindowEnforcementTest.java (85%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/MapElementsTest.java (82%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/ReduceByKeyTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/ReduceStateByKeyTest.java (91%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/ReduceWindowTest.java (81%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/SinkTest.java (80%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/SumByKeyTest.java (76%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/TopPerKeyTest.java (87%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/UnionTest.java (90%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/Util.java (79%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/WatermarkTest.java (77%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/WindowingTest.java (88%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/LongCounter.java (89%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/LongHistogram.java (89%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/NanosecondTimer.java (90%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java (88%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/SnapshotProvider.java (93%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/Snapshotable.java (91%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/accumulators/package-info.java (91%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/AbstractOperatorTest.java (89%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/ExecutorEnvironment.java (87%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/ExecutorProvider.java (93%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/ExecutorProviderRunner.java (98%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/Processing.java (96%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/junit/package-info.java (92%) rename sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/operator/test/package-info.java (85%) rename sdks/java/extensions/euphoria/euphoria-testing/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/testing/AbstractFlowTest.java (82%) rename sdks/java/extensions/euphoria/euphoria-testing/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/testing/DatasetAssert.java (97%) rename sdks/java/extensions/euphoria/{euphoria-local/src/main/java/cz/seznam/euphoria/executor/local => euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing}/package-info.java (93%) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutor.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutor.java index fca3d4424a535..c3b210420c6d7 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutor.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.VoidAccumulatorProvider; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.executor.AbstractExecutor; -import cz.seznam.euphoria.core.util.Settings; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.executor.AbstractExecutor; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; import org.apache.beam.sdk.options.PipelineOptions; import org.joda.time.Duration; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java index 69aed8e080bc6..e4df8677c649a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamExecutorContext.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java @@ -15,42 +15,43 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; - -import static java.util.stream.Collectors.toList; +package org.apache.beam.sdk.extensions.euphoria.beam; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.beam.coder.PairCoder; -import cz.seznam.euphoria.beam.io.KryoCoder; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.client.type.TypeAwareReduceFunctor; -import cz.seznam.euphoria.core.client.type.TypeAwareUnaryFunction; -import cz.seznam.euphoria.core.client.type.TypeAwareUnaryFunctor; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; -import cz.seznam.euphoria.core.util.Settings; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.beam.coder.PairCoder; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareUnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareUnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import static java.util.stream.Collectors.toList; + /** * Keeps track of mapping between Euphoria {@link Dataset} and {@link PCollection}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java index e769401c02aff..9829371b7914e 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/BeamFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java @@ -16,28 +16,29 @@ * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.beam.io.BeamWriteSink; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.VoidAccumulatorProvider; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.util.Settings; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamWriteSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.values.PCollection; + import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.values.PCollection; /** * A {@link Flow} that can be used in Euphoria operator constructions and integrates seamlessly with diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java similarity index 76% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java index f396755a5a8cd..60d830cde6517 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/DoFnCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.Context; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; +import org.apache.beam.sdk.transforms.DoFn; + +import javax.annotation.concurrent.NotThreadSafe; import java.io.Serializable; import java.util.Objects; -import javax.annotation.concurrent.NotThreadSafe; -import org.apache.beam.sdk.transforms.DoFn; /** * Collector that outputs elements to {@link BeamCollector}. diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java index c794107932830..f862f058fe12e 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlatMapTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.functional.ExtractEventTime; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; +import javax.annotation.Nullable; + class FlatMapTranslator implements OperatorTranslator { private static PCollection doTranslate( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java similarity index 78% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index b18f35d47a536..34752cfcf7c1c 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -15,29 +15,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.beam.io.BeamWriteSink; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.executor.FlowUnfolder; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; -import cz.seznam.euphoria.core.util.ExceptionUtils; -import cz.seznam.euphoria.core.util.Settings; -import java.util.IdentityHashMap; -import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamWriteSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; +import org.apache.beam.sdk.extensions.euphoria.core.util.ExceptionUtils; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; +import java.util.IdentityHashMap; +import java.util.Map; + /** * This class converts Euphoria's {@code Flow} into Beam's Pipeline. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java index 5a76677dbb27e..74ae82410f603 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/InputTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.beam.io.BeamBoundedSource; -import cz.seznam.euphoria.beam.io.BeamUnboundedSource; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.executor.FlowUnfolder; -import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamBoundedSource; +import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamUnboundedSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.values.PCollection; +import java.util.Objects; + class InputTranslator implements OperatorTranslator { private static PCollection doTranslate( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java index 2ea4901998956..1ba278e5002bb 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/LazyAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.util.Settings; import java.io.Serializable; import java.util.Objects; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java index f60f367830fb1..a184fc973b677 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; import org.apache.beam.sdk.values.PCollection; /** diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index d78b066f765fb..3e76171e4b6ac 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -15,19 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; - -import cz.seznam.euphoria.beam.window.BeamWindowFn; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import java.util.stream.StreamSupport; +package org.apache.beam.sdk.extensions.euphoria.beam; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowFn; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; @@ -39,6 +38,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import java.util.stream.StreamSupport; + /** * Translator for {@code ReduceByKey} operator. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java index 47ffb0eeb268b..7e3ae7718f922 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/ReduceStateByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; import org.apache.beam.sdk.values.PCollection; /** diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java similarity index 79% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java index c65e8df546f19..3e13e21e2f8b9 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/SingleValueCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.Context; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/TypeUtils.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/TypeUtils.java index 1bd6163fa97fe..2830fa03ace33 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/TypeUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/TypeUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/UnionTranslator.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/UnionTranslator.java index 5b3c129616071..f2786ab12cec2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/UnionTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/UnionTranslator.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java index 85bb2df1ab62b..0f10968ea5eb6 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/WrappedPCollectionOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java @@ -16,14 +16,15 @@ * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.values.PCollection; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.Datasets; -import cz.seznam.euphoria.core.client.operator.Operator; import java.util.Collection; import java.util.Collections; -import org.apache.beam.sdk.values.PCollection; /** * {@link Operator} that serves as a wrapper between a {@link PCollection} and {@link Dataset}. diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java index 4cc5e16326f04..53baa1cb23208 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/PairCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.coder; +package org.apache.beam.sdk.extensions.euphoria.beam.coder; -import cz.seznam.euphoria.core.client.util.Pair; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeParameter; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; + /** * Beam {@link StructuredCoder} for euphoria {@link Pair}. * @param diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/package-info.java index ebfcb8688bea6..74215eca1e3e5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/coder/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/package-info.java @@ -19,4 +19,4 @@ /** * Beam coders. */ -package cz.seznam.euphoria.beam.coder; +package org.apache.beam.sdk.extensions.euphoria.beam.coder; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java index 35d0a43371e51..274800d994c4a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.io; +package org.apache.beam.sdk.extensions.euphoria.beam.io; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedDataSource; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; -import cz.seznam.euphoria.core.client.io.BoundedDataSource; import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.stream.Collectors; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; /** * A {@link BoundedSource} created from {@link BoundedDataSource}. @@ -59,7 +60,7 @@ public long getEstimatedSizeBytes(PipelineOptions po) throws Exception { @Override public BoundedReader createReader(PipelineOptions po) throws IOException { - final cz.seznam.euphoria.core.client.io.BoundedReader reader = wrap.openReader(); + final org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedReader reader = wrap.openReader(); return new BoundedReader() { private T current = null; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java index 80bef5d35f9d3..6e88d96d49a21 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamUnboundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java @@ -15,9 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.io; +package org.apache.beam.sdk.extensions.euphoria.beam.io; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedDataSource; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; -import cz.seznam.euphoria.core.client.io.UnboundedDataSource; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -25,10 +30,6 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.joda.time.Instant; /** * A {@link UnboundedSource} created from {@link UnboundedDataSource}. @@ -96,7 +97,7 @@ public List>> split( public UnboundedReader createReader( PipelineOptions options, BeamCheckpointMark checkpointMark) throws IOException { - final cz.seznam.euphoria.core.client.io.UnboundedReader reader; + final org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedReader reader; reader = wrap.getPartitions().get(partitionId).openReader(); return new UnboundedReader() { diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java index f0a64a2d8ad30..630af91c93638 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/BeamWriteSink.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java @@ -15,12 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.io; +package org.apache.beam.sdk.extensions.euphoria.beam.io; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.Writer; -import java.io.IOException; -import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Writer; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -28,6 +26,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import java.io.IOException; +import java.util.Objects; + /** * Write to output sink using beam. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java index 4bc925c14a91a..33a2bf148d496 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/KryoCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java @@ -15,12 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.io; +package org.apache.beam.sdk.extensions.euphoria.beam.io; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; -import cz.seznam.euphoria.core.client.functional.VoidFunction; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.VoidFunction; +import org.objenesis.strategy.StdInstantiatorStrategy; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -31,8 +34,6 @@ import java.io.ObjectOutputStream; import java.io.OutputStream; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.sdk.coders.CustomCoder; -import org.objenesis.strategy.StdInstantiatorStrategy; /** * Coder using Kryo as (de)serialization mechanism. TODO: we should drop this class entirely diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/package-info.java index 5bdc183ca6202..df54fb38385c5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/io/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/package-info.java @@ -19,4 +19,4 @@ /** * IO centered classes. */ -package cz.seznam.euphoria.beam.io; +package org.apache.beam.sdk.extensions.euphoria.beam.io; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/package-info.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/package-info.java index 6b4918a8dc1ac..5e07349b3c639 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/package-info.java @@ -19,4 +19,4 @@ /** * Euphoria to Beam translation package. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindow.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindow.java index a85d8c9b973a5..68584e0487799 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindow.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.joda.time.Instant; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java index 649f29ddf320b..d895646034e6c 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java @@ -15,23 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; -import cz.seznam.euphoria.beam.io.KryoCoder; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import java.util.Collection; -import java.util.Objects; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import java.util.Collection; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + /** * A {@code WindowFn} wrapper of {@code Windowing}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java index 3f48e29f96c8a..ca4637c3269a4 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; import java.util.Objects; /** diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/package-info.java index 70681786cfceb..60d34167d74cd 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/package-info.java @@ -19,4 +19,4 @@ /** * Windowing centered classes. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java index 6db2c07c64600..cd2ed5ecb30af 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/BeamFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java @@ -16,27 +16,23 @@ * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.CountByKey; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.testing.DatasetAssert; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.Serializable; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; @@ -48,6 +44,11 @@ import org.junit.Ignore; import org.junit.Test; +import java.io.Serializable; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; + /** * Test for {@link BeamFlow}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java index 0c4049b9e4a69..7e30be17196c6 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java @@ -15,19 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.testing.DatasetAssert; -import java.util.Arrays; -import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.junit.Test; +import java.util.Arrays; +import java.util.concurrent.ExecutionException; + /** * Test {@code FlatMap} operator's integration with beam. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java similarity index 78% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index 116676cadda37..d7835a4897ec1 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -15,44 +15,45 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam; - -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.CountTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.testing.DatasetAssert; -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; +package org.apache.beam.sdk.extensions.euphoria.beam; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.junit.Ignore; import org.junit.Test; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; + +import static org.junit.Assert.assertTrue; + /** * Simple test suite for RBK. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java similarity index 81% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java index 94c865175c07f..9fee23eeba998 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.testkit; +package org.apache.beam.sdk.extensions.euphoria.beam.testkit; -import cz.seznam.euphoria.beam.BeamExecutor; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.operator.test.junit.ExecutorEnvironment; -import cz.seznam.euphoria.operator.test.junit.ExecutorProvider; -import java.time.Duration; +import org.apache.beam.sdk.extensions.euphoria.beam.BeamExecutor; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorEnvironment; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import java.time.Duration; + /** * Local {@link ExecutorProvider}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java similarity index 76% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java index 6deed8a99790d..a8d27582d8937 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.testkit; +package org.apache.beam.sdk.extensions.euphoria.beam.testkit; -import cz.seznam.euphoria.operator.test.FlatMapTest; -import cz.seznam.euphoria.operator.test.UnionTest; -import cz.seznam.euphoria.operator.test.junit.ExecutorProvider; -import cz.seznam.euphoria.operator.test.junit.ExecutorProviderRunner; +import org.apache.beam.sdk.extensions.euphoria.operator.test.FlatMapTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.UnionTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProviderRunner; import org.junit.runner.RunWith; import org.junit.runners.Suite; /** - * This is a copy of {@link cz.seznam.euphoria.operator.test.AllOperatorsSuite} to allow us track + * This is a copy of {@link org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite} to allow us track * progress on incrementally implementing operator and their tests. TODO: When done, this class * should go away and original should be used instead */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsTest.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsTest.java index 8021ca628983d..33e8b246d35d6 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.testkit; +package org.apache.beam.sdk.extensions.euphoria.beam.testkit; /** * TODO: write javadoc. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/package-info.java deleted file mode 100644 index d0f22a4b3e29a..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.dataset.windowing; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/package-info.java deleted file mode 100644 index efe5773695af9..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.flow; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/package-info.java deleted file mode 100644 index d47ffc9b6d119..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.functional; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/package-info.java deleted file mode 100644 index d3ddf8a27145b..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.io; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/package-info.java deleted file mode 100644 index 0a0694d9ed96d..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.lib; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/package-info.java deleted file mode 100644 index 55d69605698e0..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.operator.hint; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/package-info.java deleted file mode 100644 index 912042d42ba05..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.operator; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/package-info.java deleted file mode 100644 index 37292ae7d3760..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.operator.state; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/package-info.java deleted file mode 100644 index bbe0848752bed..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.triggers; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/package-info.java deleted file mode 100644 index 3bd83dba7fe6a..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.type; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/package-info.java deleted file mode 100644 index f77dff253a83e..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/package-info.java deleted file mode 100644 index c49d9db15ad5b..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.executor.graph; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/package-info.java deleted file mode 100644 index b6c430e18442a..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.executor.greduce; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/package-info.java deleted file mode 100644 index 819c90c0687bd..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.executor.io; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/package-info.java deleted file mode 100644 index 764c526e002de..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.executor; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/package-info.java deleted file mode 100644 index 4dd714da9ff46..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.executor.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/package-info.java deleted file mode 100644 index b78b11bec69be..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.time; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/package-info.java deleted file mode 100644 index e5c7f9b64ee29..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/package-info.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ -/** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java index 874d989033c51..02d90bd0ddc5a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/Audience.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/Audience.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.audience; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.audience; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/package-info.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java index 842e92460883d..a1d9470ef242d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/audience/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/audience/package-info.java @@ -17,4 +17,4 @@ */ /** Intended audience of API. */ -package cz.seznam.euphoria.core.annotation.audience; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.audience; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java index 43ba65f65c9a7..600cc339b4dd0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Basic.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Basic.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.operator; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java index c9fe25b299bf8..017ead756e6fe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Derived.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Derived.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.operator; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java index e4a96d8ecd2a9..1d738defcaedf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/Recommended.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/Recommended.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.operator; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java index 4bd9b6419da82..60506045b12d0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/StateComplexity.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/StateComplexity.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.operator; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator; /** Space complexity of an operator's state in each window depending on the size of input. */ public enum StateComplexity { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java new file mode 100644 index 0000000000000..225e25837e2ef --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/operator/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.annotation.operator; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java index bcd57a1073acb..ec4d3e17e43ca 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/Experimental.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/Experimental.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.annotation.stability; +package org.apache.beam.sdk.extensions.euphoria.core.annotation.stability; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java new file mode 100644 index 0000000000000..934a4eedb6ee6 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/annotation/stability/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.annotation.stability; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java index 95104409244ab..4a5fbb9c78bdf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Accumulators collect values from user functions. Accumulators allow user to calculate statistics diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java index 377af190a35a4..5a75734f820cc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/AccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.util.Settings; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java index 25da368aff42a..b5e93ef5d3b9d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Counter is a type of accumulator making a sum from integral numbers. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java index 0c345c34c5143..18c71ede8a74c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Histogram.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Histogram.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Histogram is a type of accumulator recording a distribution of different values. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java index be439c11f2807..dde50ab2c7f30 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Timer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.time.Duration; import java.util.concurrent.TimeUnit; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java index fe29bd0809760..5afa39dfa1bc4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/VoidAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.util.Settings; -import java.time.Duration; -import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicBoolean; + /** * Placeholder implementation of {@link AccumulatorProvider} that may be used in executors as a * default. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java new file mode 100644 index 0000000000000..e65e600e2fa9f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java index e87a4f5e029b5..2fcd89dfe1e5e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; + +import javax.annotation.Nullable; import java.io.Serializable; import java.util.Collection; -import javax.annotation.Nullable; /** * A dataset abstraction. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Datasets.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Datasets.java index 67f399e09d847..4bdcc952218af 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/Datasets.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Datasets.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; /** Various dataset related utils. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java index 9c5e24c1e73ba..9e2a99d0464f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/InputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.Operator; -import java.util.Collection; import javax.annotation.Nullable; +import java.util.Collection; /** {@code InputDataset} that is input of a {@code Flow}. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java similarity index 81% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java index 55b1cdf50157f..4b27e62034048 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/OutputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.Operator; -import java.util.Collection; import javax.annotation.Nullable; +import java.util.Collection; /** {@code OutputDataset} that is output of some operator. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java new file mode 100644 index 0000000000000..f5cf74915c83f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java index 156e17c96addd..4c01a32b07723 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Count.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import static java.util.Collections.singleton; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.CountTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; +import static java.util.Collections.singleton; /** Count tumbling windowing. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java index 7174c702b7e99..5eec4e2f40566 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/GlobalWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.NoopTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; import java.io.ObjectStreamException; import java.util.Collections; @@ -63,7 +64,7 @@ public int hashCode() { /** Singleton window. */ public static final class Window - extends cz.seznam.euphoria.core.client.dataset.windowing.Window { + extends org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window { static final Window INSTANCE = new Window(); diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java index 2db906ff8e2c3..e41f3d16909a0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/MergingWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.util.Pair; import java.util.Collection; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java index 121bf277b0c25..c085db6b85489 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Session.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.AfterFirstCompositeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.PeriodicTimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.stability.Experimental; -import cz.seznam.euphoria.core.client.triggers.AfterFirstCompositeTrigger; -import cz.seznam.euphoria.core.client.triggers.PeriodicTimeTrigger; -import cz.seznam.euphoria.core.client.triggers.TimeTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.util.Pair; +import javax.annotation.Nullable; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -34,7 +34,8 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; -import javax.annotation.Nullable; + +import static com.google.common.base.Preconditions.checkArgument; /** Session windowing. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java index 70be1d36d7928..b55c70b9ba39a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Time.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java @@ -15,20 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Collections.singleton; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.AfterFirstCompositeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.PeriodicTimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.AfterFirstCompositeTrigger; -import cz.seznam.euphoria.core.client.triggers.PeriodicTimeTrigger; -import cz.seznam.euphoria.core.client.triggers.TimeTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; +import javax.annotation.Nullable; import java.time.Duration; import java.util.Arrays; import java.util.Objects; -import javax.annotation.Nullable; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Collections.singleton; /** Time based tumbling windowing. Windows can't overlap. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeInterval.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeInterval.java index 3e93b40f0b875..f81b793e31905 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeInterval.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeInterval.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** TODO: complete javadoc. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java index e5269a82ebf0c..bc09047e1e462 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSliding.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java @@ -15,19 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; - -import static com.google.common.base.Preconditions.checkArgument; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; import com.google.common.collect.AbstractIterator; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.triggers.TimeTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; + import java.time.Duration; import java.util.Iterator; import java.util.Objects; +import static com.google.common.base.Preconditions.checkArgument; + /** Time sliding windowing. */ @Audience(Audience.Type.CLIENT) public final class TimeSliding implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java index 53ca8d7544cb1..4e731bfe14d65 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Window.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowedElement.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowedElement.java index 95a5fec2f9b17..1bbdc5ace9b9e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowedElement.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A single data element flowing in dataset. Every such element is associated with a window diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java index 0a2ded8352633..e810d579bf448 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/windowing/Windowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.Trigger; import java.io.Serializable; /** A windowing policy of a dataset. All implementations must implement equals/hashCode. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/package-info.java new file mode 100644 index 0000000000000..2f833c0cf8855 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java index 09e30e9f4de1b..7357b3d2274bb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java @@ -15,16 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.flow; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.Datasets; -import cz.seznam.euphoria.core.client.functional.ExtractEventTime; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.util.Settings; +package org.apache.beam.sdk.extensions.euphoria.core.client.flow; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; import java.io.IOException; import java.io.ObjectOutputStream; import java.io.OutputStream; @@ -38,9 +42,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** A dependency graph of operators. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Util.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Util.java index 8a5063a1a959d..774058abc78ef 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Util.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.flow; +package org.apache.beam.sdk.extensions.euphoria.core.client.flow; import javax.annotation.Nullable; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/package-info.java new file mode 100644 index 0000000000000..3063643c3f59b --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.flow; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java index 018ec4e0cb461..ffc91f8d3e5dd 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java index b98272da45e3a..eff3305762b43 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Collector; import java.io.Serializable; /** Functor of two arguments. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java index 253c10d84f5c1..656b7dd389562 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Function of two arguments of the same type. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java index 5a7b0590dd9df..41a6ae98f4d4b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Function reducing stream of elements into single one of the same type. The applied function has diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java index 4e712cbbf3799..b5e086ae79170 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CompositeUnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CompositeUnaryFunction.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A function that is composition of two unary functions. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java index 700b618da5aff..86b8813eb5dd6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** A consumer of given type. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java index 6343fc6c7f30c..3f95a90a67f32 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** @param */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java index e2fb55f6cc80e..988bedf3c1391 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.util.stream.Stream; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java index 22afa5fe65dd1..721d777343057 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.util.stream.Stream; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java index a46769daf0e74..adcdfbff3d74b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Function of two arguments of the same type. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java index f30bb3a556437..b712f7b55efb4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** Function taking three arguments. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java index 778bb542eb398..0d742c87d59d6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.type.TypeHint; import java.io.Serializable; /** @param */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java index 2de5a936fc87f..5fe69656eeb54 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java index 4966643f120cb..3b3953839ea21 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctionEnv.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Context; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java index 7aebe04ee7559..3292b4fb94f2c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Collector; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java index 6a07ff82f3e0a..e2115ebc5dc4c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Binary function of X returning boolean. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java index 5c4c80a2d8136..1862726abf2ec 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.functional; +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** Function taking zero arguments. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java new file mode 100644 index 0000000000000..4593a726cc2fb --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.functional; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java index 85aacc78447b4..12e41c44af3d5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.IOException; import java.util.List; import java.util.Set; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java index cca8ce4c2a76d..e93fab03e5607 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java index 72632b346125b..f32e65c964819 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Closeable; import java.util.Iterator; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java index 4b9d93c90fdac..f990ff43af0da 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Collector.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Collector.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Extends {@link Environment} with write capability. Used in user defined functors. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java index 1d850bad7fecb..8ea8801d8b45f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Used in user defined functions to access environment methods. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java index 4ad9eea3c9f88..b8a01fd8650bb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; import java.io.IOException; import java.io.Serializable; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java index 2a476f032b9e4..d493c240409f1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSinks.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.MapElements; import java.io.IOException; /** Various {@link DataSink} related utilities. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java index eb04d1f8c9342..ae86930ca38bc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/DataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** Source of data for dataset. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java similarity index 79% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java index 055d34540b733..af7b8faa02478 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Environment.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Environment.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** Defines basic methods available in user defined functions. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java index 579e74eab63d1..6f97645a6faca 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Closeable; /** An {@code Iterable} that is externalized to external storage (i.e. spilled to disk). */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java index 152a0adb1bd74..6ac12de28223b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java @@ -15,11 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.Consumer; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.Consumer; + +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -28,7 +30,6 @@ import java.util.Map; import java.util.WeakHashMap; import java.util.stream.Collectors; -import javax.annotation.Nullable; /** A data sink that stores data in list. */ @Audience({Audience.Type.CLIENT, Audience.Type.TESTS}) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java index 9684bc1ffa751..aa9d763903387 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ListDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; import com.google.common.collect.Lists; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + import java.io.IOException; import java.time.Duration; import java.util.ArrayList; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java index c929299013643..260b60269d063 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/MultiDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.util.IOUtils; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java index 2ebee541619a1..1a43c05eb21f0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/SpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; import com.google.common.collect.Iterables; import com.google.common.io.Closeables; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.util.IOUtils; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; + import java.io.IOException; import java.io.Serializable; import java.util.Collection; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java index 1cbebc73edede..0ab7a88fe2a56 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/StdoutSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.IOException; import java.io.PrintStream; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java index c5717bbfc8d4e..fa615129d6524 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; import java.util.List; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java index 46feaff451811..474467fd24b24 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedPartition.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.IOException; import java.io.Serializable; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedReader.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedReader.java index fe46cb792a521..e7cbd9035a637 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnboundedReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedReader.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A reader of data from unbounded partition. The reader differs from {@code BoundedReader} by the diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java index 93aaf155a2362..a918b5d17d7a5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/UnsplittableBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java @@ -21,9 +21,10 @@ * and open the template in the editor. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.util.Collections; import java.util.List; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java index cc6b6d958abb4..440a5ff5b8645 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/VoidSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.IOException; /** TODO: complete javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java index 1370b7bc13e04..3fa85b27ba1d7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Writer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Closeable; import java.io.IOException; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/package-info.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java index 130baac3ca2c2..af5435a4bb1ea 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/dataset/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ /** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.dataset; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java index 2737fae2f7981..6225db81b7879 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/lib/Split.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.lib; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Derived; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.UnaryPredicate; -import cz.seznam.euphoria.core.client.operator.Filter; +package org.apache.beam.sdk.extensions.euphoria.core.client.lib; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; + import java.io.Serializable; import java.util.Objects; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java new file mode 100644 index 0000000000000..55eae6f176bc0 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.lib; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java index 1d535e79bbe36..c1deb2d9101f1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/AssignEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Derived; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.ExtractEventTime; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + import java.util.Objects; import java.util.Set; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Builders.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Builders.java index cc96a7a8f2019..1322b68c6a576 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Builders.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Builders.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.type.TypeAwareUnaryFunction; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareUnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** * Common methods used in operator builders to share related javadoc descriptions. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java index ebee354cb51cb..56e395d6a42ed 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/CountByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java @@ -15,23 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Derived; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + +import javax.annotation.Nullable; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; /** * Operator counting elements with same key. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java index ac9df09713dba..96c51fd92d0bf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Distinct.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java @@ -15,25 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Recommended; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + +import javax.annotation.Nullable; import java.util.Collections; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; /** * Operator outputting distinct (based on {@link Object#equals}) elements. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java similarity index 78% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java index e469d6ea037c6..2c5546fd589ee 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ElementWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; import java.util.Set; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java index 4614ac518c7fa..a57ef4085c26e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Filter.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Derived; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryPredicate; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + import java.util.Objects; import java.util.Set; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java index 80c7d4d79e9f6..28c84bfde9d7c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FlatMap.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java @@ -15,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Basic; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.ExtractEventTime; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; + +import javax.annotation.Nullable; import java.util.Collections; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; /** * A transformation of a dataset from one type into another allowing user code to generate zero, diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java index 0f43ae164b352..c2dd55345787e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/FullJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; + import java.util.Objects; import java.util.Optional; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java index 2c57ddb62cc08..d819f36108187 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Join.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java @@ -15,35 +15,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Recommended; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.annotation.stability.Experimental; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; -import cz.seznam.euphoria.core.client.util.Either; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Either; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + +import javax.annotation.Nullable; import java.util.Arrays; import java.util.Collection; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; /** * Inner join of two datasets by given key producing single new dataset. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java index e6ad99c76d1f8..35d27fecb54c0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/LeftJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; + import java.util.Objects; import java.util.Optional; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java index dd5f607b1f57a..8cf6e28edc348 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/MapElements.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Derived; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctionEnv; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctionEnv; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; + import java.util.Collections; import java.util.Objects; import java.util.Set; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java index 1db1f410791fe..bf74ed5eb31cc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/Operator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java @@ -15,20 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.Datasets; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; import java.io.Serializable; import java.util.Collection; import java.util.Set; +import static com.google.common.base.Preconditions.checkArgument; + /** An operator base class. All operators inherit his class. */ @Audience(Audience.Type.INTERNAL) public abstract class Operator implements Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java index e5f1858081299..662ed44bf5569 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/OptionalMethodBuilder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; import java.util.Objects; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java index a9cf5d2a04233..3f20ab4170768 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java @@ -15,45 +15,46 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Recommended; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; -import cz.seznam.euphoria.core.client.functional.ReduceFunction; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ExternalIterable; -import cz.seznam.euphoria.core.client.io.SpillTools; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StateFactory; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.type.TypeAwareReduceFunctor; -import cz.seznam.euphoria.core.client.type.TypeAwareUnaryFunction; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.util.SingleValueContext; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateFactory; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareUnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; + +import javax.annotation.Nullable; import java.util.Comparator; import java.util.Objects; import java.util.Set; import java.util.stream.Stream; import java.util.stream.StreamSupport; -import javax.annotation.Nullable; /** * Operator performing state-less aggregation by given reduce function. The reduction is performed diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java index 4221ae73c0749..3c1fab841b56c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java @@ -15,27 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.operator.Basic; -import cz.seznam.euphoria.core.annotation.operator.StateComplexity; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateFactory; -import cz.seznam.euphoria.core.client.operator.state.StateMerger; -import cz.seznam.euphoria.core.client.type.TypeAwareUnaryFunction; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateFactory; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateMerger; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeAwareUnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; + +import javax.annotation.Nullable; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; /** * A {@link ReduceStateByKey} operator is a stateful, complex, lower-level-api, but very powerful @@ -297,7 +298,7 @@ public static class DatasetBuilder4 the type of input elements for the states diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/Storage.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/Storage.java index 86782e2e9b821..e7be21148d94d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/Storage.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator.state; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A storage for state data. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java index 7ece4c26acd45..5fb51455e360a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator.state; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** Descriptor of storage. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java index 081040c520673..2ec8e68a9b7fb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator.state; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; /** A provider of storage instances. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorage.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorage.java index 37f8ecff99f8a..bdb2a9dbcae2a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorage.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator.state; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * State storage for single value. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorageDescriptor.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorageDescriptor.java index b4cc105d5f77c..3579c54f45212 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorageDescriptor.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator.state; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; /** * Descriptor of {@code ValueStorage}. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/package-info.java new file mode 100644 index 0000000000000..cc316044ac4af --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java index 03a2b0c235b04..398b80a838402 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; import java.util.List; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/CountTrigger.java similarity index 81% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/CountTrigger.java index 77819792d0621..40e4a76866bec 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/CountTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/CountTrigger.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; /** A {@link Trigger} that fires once the count of elements reaches given count. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/NoopTrigger.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/NoopTrigger.java index 157d98fe8e06b..fa1828aca5b27 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/NoopTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/NoopTrigger.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** A trigger implementation which actually never fires any of the observed windows. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/PeriodicTimeTrigger.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/PeriodicTimeTrigger.java index e615283c308c0..aa5e8b013f312 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/PeriodicTimeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/PeriodicTimeTrigger.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TimeTrigger.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TimeTrigger.java index 0f13f502d906a..1439f733ed283 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TimeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TimeTrigger.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java index 01338a19d5653..bb6d9e0533261 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/Trigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.StorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; import java.io.Serializable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TriggerContext.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TriggerContext.java index 0e77739a009ee..d457fa1bcb6c1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/triggers/TriggerContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/TriggerContext.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.triggers; +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.operator.state.StorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; /** A context is given to {@link Trigger} methods to allow them to register timer callbacks. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/package-info.java new file mode 100644 index 0000000000000..d144ee161c251 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java index 434aa26f02dbe..d4f45d272b78a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/AbstractTypeAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.type; +package org.apache.beam.sdk.extensions.euphoria.core.client.type; + +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.TypeHintAware; -import cz.seznam.euphoria.core.client.functional.TypeHintAware; import java.io.Serializable; import java.util.Objects; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java index 6badb6a3c844d..f2f0145b588c9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.type; +package org.apache.beam.sdk.extensions.euphoria.core.client.type; + +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.io.Collector; import java.util.stream.Stream; /** TODO: complete javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunction.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunction.java index 992b18248e146..647dff4c33c54 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunction.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.type; +package org.apache.beam.sdk.extensions.euphoria.core.client.type; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; /** TODO: complete javadoc. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunctor.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunctor.java index 2bbb5190b6f56..dd95131a59c9d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeAwareUnaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareUnaryFunctor.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.type; +package org.apache.beam.sdk.extensions.euphoria.core.client.type; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; /** TODO: complete javadoc. */ public class TypeAwareUnaryFunctor diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java index 3cb6afb0a16aa..ed05d3200a614 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/type/TypeHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.type; +package org.apache.beam.sdk.extensions.euphoria.core.client.type; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; + import java.io.Serializable; import java.lang.reflect.Type; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java new file mode 100644 index 0000000000000..0160c7b2e082f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.type; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java index 72b7c04f714c5..441cab1eefb8e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Either.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import static java.util.Objects.requireNonNull; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import javax.annotation.Nullable; +import static java.util.Objects.requireNonNull; + /** Either LeftT or RightT element. */ @Audience(Audience.Type.INTERNAL) public final class Either { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java index 36ffc608de93b..2c8c9f406528f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Fold.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; + +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.executor.util.SingleValueContext; import java.io.Serializable; import java.util.stream.Stream; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java index 6b013f8eece8a..8cb5eeeb861e2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Max.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; import java.util.Comparator; /** Calculate maximum. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java index 543db17204c73..c0b04a1cdaccc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Pair.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.util.Comparator; import java.util.Objects; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java index 812cfd415ba12..bf1f2a0fe9d16 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Sums.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; import java.util.stream.Collectors; /** Provides commonly used function objects around computing sums. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java index 533a524882359..385c1d40bd60e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/util/Triple.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.util.Objects; /** Triple of any types. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java new file mode 100644 index 0000000000000..f467be9df43dd --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.client.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java index 028274be91f33..36f17d0b22b79 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/AbstractExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; + +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import cz.seznam.euphoria.core.client.flow.Flow; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** TODO: complete javadoc. */ public abstract class AbstractExecutor implements Executor { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Constants.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Constants.java index 37923ee55354c..7b930d1775703 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Constants.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Constants.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Various constants (mostly) related to configuration keys independent of executors. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java similarity index 79% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java index 465f778a3f2bc..6ab6ac5fd5232 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/Executor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.VoidAccumulatorProvider; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; + import java.util.Set; import java.util.concurrent.CompletableFuture; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java index 749f4518a9b8e..c3e4c3817e0e7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowUnfolder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryPredicate; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; + import java.util.ArrayList; import java.util.Collection; import java.util.Collections; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java index 032eab9161dbe..c489bb6b54144 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/FlowValidator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java @@ -15,26 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; -import static com.google.common.base.Preconditions.checkState; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.WindowWiseOperator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.WindowingRequiredException; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.WindowWiseOperator; -import cz.seznam.euphoria.core.client.operator.WindowingRequiredException; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkState; + /** Validate invariants. Throw exceptions if any invariant is violated. */ class FlowValidator { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java index 61cb5506bc603..7287f6094d702 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/VectorClock.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; +package org.apache.beam.sdk.extensions.euphoria.core.executor; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java similarity index 97% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java index 26bb969cf2e95..80782292eb661 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/DAG.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.graph; +package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.util.Pair; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java index 8deb1305c3027..3d7dfe5a50b84 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/graph/Node.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.graph; +package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; -import javax.annotation.Nullable; /** A single Node in DAG. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/package-info.java new file mode 100644 index 0000000000000..e15f6725603c9 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java index 248756c96ac5e..815d115ec3d06 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/GroupReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java @@ -15,41 +15,42 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.greduce; +package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.MergingStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateFactory; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateMerger; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.Storage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import static com.google.common.base.Preconditions.checkState; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; -import cz.seznam.euphoria.core.client.io.Context; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.MergingStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StateFactory; -import cz.seznam.euphoria.core.client.operator.state.StateMerger; -import cz.seznam.euphoria.core.client.operator.state.Storage; -import cz.seznam.euphoria.core.client.operator.state.StorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.util.Pair; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Objects; +import static com.google.common.base.Preconditions.checkState; + /** * An implementation of a RSBK group reducer of an ordered stream of already grouped (by a specific * key) and windowed elements where no late-comers are tolerated. Use this class only in batch mode! @@ -264,7 +265,7 @@ public interface WindowedElementFactory { WindowedElement create(W window, long timestamp, T element); } - class ElementCollector implements Context, cz.seznam.euphoria.core.client.io.Collector { + class ElementCollector implements Context, org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector { final Collector>> out; final WidT window; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java index 8067210fb290d..c8c7822d578d5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TimerSupport.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.greduce; +package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; + +import javax.annotation.Nullable; import java.util.HashSet; import java.util.PriorityQueue; -import javax.annotation.Nullable; /** * Keeps track of the current watermark within a reduce operation and provides services around diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java index 61add66a7a899..593787c15ac20 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/greduce/TriggerStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.greduce; - -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.Storage; -import cz.seznam.euphoria.core.client.operator.state.StorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; +package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.Storage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; + import java.util.HashMap; import java.util.Objects; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/package-info.java new file mode 100644 index 0000000000000..c74ebeba2bf68 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java index 1a5ec7c2a40db..7d0b0f828213f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.ExternalIterable; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java index e3a33a10b53c9..ce31cd4e8d4e1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/GenericSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java @@ -15,14 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; import com.google.common.annotations.VisibleForTesting; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.ExternalIterable; -import cz.seznam.euphoria.core.client.io.SpillTools; -import cz.seznam.euphoria.core.executor.Constants; -import cz.seznam.euphoria.core.util.Settings; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Constants; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.util.ArrayList; import java.util.Collection; @@ -31,8 +34,6 @@ import java.util.UUID; import java.util.stream.Stream; import java.util.stream.StreamSupport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** An implementation of {@code SpillTools} to be used by executors. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java index f48921599c91d..dc6e01c57dce2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SerializerFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Closeable; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java index 6ab078ba39b14..3fa923867c8f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/io/SpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.File; import java.io.Serializable; import java.net.URI; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/package-info.java new file mode 100644 index 0000000000000..89bd18fdb3b5a --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/package-info.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/package-info.java index a4a0c6e6abd7c..95f062f571de1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ /** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.client.accumulators; +package org.apache.beam.sdk.extensions.euphoria.core.executor; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java index a66b81825f34c..f55e88935a077 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/InMemExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.util; +package org.apache.beam.sdk.extensions.euphoria.core.executor.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.ExternalIterable; import java.util.Iterator; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java index 3bd36d8e980cf..4d791ab50fff5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/MultiValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java @@ -15,19 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.util; +package org.apache.beam.sdk.extensions.euphoria.core.executor.util; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.Context; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; + +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import javax.annotation.Nullable; /** TODO: complete javadoc. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java similarity index 73% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java index b2c6a91fb7b6c..77e22a4f62a79 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.util; +package org.apache.beam.sdk.extensions.euphoria.core.executor.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join.Type; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.Join.Type; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.hint.SizeHint; import java.util.ArrayList; /** Util class when specific executors use the same methods for operator translation. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java index b68b3ab44d8e5..3e75d02875e3f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/executor/util/SingleValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.util; +package org.apache.beam.sdk.extensions.euphoria.core.executor.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.Context; import javax.annotation.Nullable; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/package-info.java new file mode 100644 index 0000000000000..501c050549439 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/package-info.java @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ +/** TODO: complete javadoc. */ +package org.apache.beam.sdk.extensions.euphoria.core.executor.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java index c273aec24d488..580ab9d8b85f6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.time.Duration; /** TODO: complete javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java index 312d138295e2c..7a65959689bc2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.Serializable; import java.time.Duration; import java.util.Date; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviderAware.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviderAware.java index e7b9e01ff92f3..4974c7d9dd842 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviderAware.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; -import cz.seznam.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** TODO: complete javadoc. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java index 4c73563ba6d74..71a3eb43014d7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviders.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.time.Duration; import java.util.Calendar; import java.util.Date; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java index b568cadb4600b..0bf908572f3f4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.time.Duration; import java.util.Timer; import java.util.TimerTask; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A scheduler implementation with an underlying timer periodically executing the specified diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/package-info.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/package-info.java index 846802a790035..e00d711e0776f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/operator/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ /** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.annotation.operator; +package org.apache.beam.sdk.extensions.euphoria.core.time; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/ExceptionUtils.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/ExceptionUtils.java index 7129c7d7e54cf..3ef6a4eb58416 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/ExceptionUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/ExceptionUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.util; +package org.apache.beam.sdk.extensions.euphoria.core.util; import java.util.function.Consumer; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java similarity index 97% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java index e1144dbd16d69..deaeee69c0ece 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/IOUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/IOUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.util; +package org.apache.beam.sdk.extensions.euphoria.core.util; import java.io.IOException; import java.util.stream.Stream; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java index 12e5dfc903b71..093ae93e5db61 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/InstanceUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.util; +package org.apache.beam.sdk.extensions.euphoria.core.util; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.lang.reflect.Constructor; /** diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java index 1d4e3999f0c8c..32559d520628c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/util/Settings.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.util; +package org.apache.beam.sdk.extensions.euphoria.core.util; -import static java.util.Objects.requireNonNull; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.util.Pair; +import javax.annotation.Nullable; import java.io.Serializable; import java.net.URI; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -import javax.annotation.Nullable; + +import static java.util.Objects.requireNonNull; /** * General utility class to store key/value pairs as strings providing converter methods for diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/package-info.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java index 9fa90a5fab118..9d575d2710eb7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/annotation/stability/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/package-info.java @@ -16,4 +16,4 @@ * limitations under the License. */ /** TODO: complete javadoc. */ -package cz.seznam.euphoria.core.annotation.stability; +package org.apache.beam.sdk.extensions.euphoria.core.util; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java index 339a97fea0b78..e79d1b3ce54b2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/SessionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; + import java.time.Duration; import java.util.Arrays; import java.util.Collection; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** TODO: complete javadoc. */ public class SessionTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java index 61818390547a3..ce31d2da05656 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeSlidingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; - -import static org.junit.Assert.assertEquals; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import java.time.Duration; import org.junit.Test; +import java.time.Duration; + +import static org.junit.Assert.assertEquals; + /** TODO: complete javadoc. */ public class TimeSlidingTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java index 8a24c1d072664..6d591eedae822 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimeTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; - -import static org.junit.Assert.assertEquals; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; import com.google.common.collect.Iterables; -import java.time.Duration; import org.junit.Test; +import java.time.Duration; + +import static org.junit.Assert.assertEquals; + /** TODO: complete javadoc. */ public class TimeTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimestampedElement.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimestampedElement.java index a6193dd195137..8fc6ff8759ff2 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/TimestampedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimestampedElement.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; /** TODO: complete javadoc. */ class TimestampedElement implements WindowedElement { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java index 8e7c818caacd7..605fa0a2ee29b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/dataset/windowing/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java @@ -15,21 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.dataset.windowing; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; + import java.time.Duration; import java.util.Arrays; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.junit.Test; + +import static org.junit.Assert.*; /** TODO: complete javadoc. */ public class WindowingTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java index 1d7d1ea1728d7..f475cff102cf8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/flow/TestFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java @@ -15,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.flow; +package org.apache.beam.sdk.extensions.euphoria.core.client.flow; -import static org.junit.Assert.assertEquals; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.MockStreamDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.junit.Before; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.io.MockStreamDataSource; -import cz.seznam.euphoria.core.client.operator.Filter; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.Union; import java.util.Arrays; import java.util.Collection; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.junit.Before; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; /** Test some basic features of flow. */ public class TestFlow { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/EmptyReader.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/EmptyReader.java index 3463e38daaad6..0c80e5957ebf1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/EmptyReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/EmptyReader.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; import java.io.IOException; import java.util.NoSuchElementException; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java index 591a2d827945f..8116c17ded672 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/ListDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; - -import static org.junit.Assert.assertEquals; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.testing.DatasetAssert; +import org.apache.beam.sdk.extensions.euphoria.core.testing.DatasetAssert; import org.junit.Test; +import static org.junit.Assert.assertEquals; + /** TODO: complete javadoc. */ public class ListDataSinkTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MockStreamDataSource.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MockStreamDataSource.java index cf27674bedb27..e677c2e905302 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MockStreamDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MockStreamDataSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; import java.io.Serializable; import java.util.ArrayList; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java index 5c6438ab87d06..c363019ba2e52 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/io/MultiDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.io; +package org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import java.io.IOException; -import org.junit.Test; -import org.mockito.Mockito; - /** Tests behavior of {@link MultiDataSink}. */ public class MultiDataSinkTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java index 8c61a67af7355..8c18276dabafa 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/lib/SplitTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java @@ -15,22 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.lib; +package org.apache.beam.sdk.extensions.euphoria.core.client.lib; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Util; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryPredicate; -import cz.seznam.euphoria.core.client.operator.Filter; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.Util; import java.util.Optional; -import org.junit.Test; + +import static org.junit.Assert.*; /** TODO: complete javadoc. */ public class SplitTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java index 71f5a0213b10f..be6d5daa80a14 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java @@ -15,19 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.util.Pair; import java.time.Duration; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator CountByKey. */ public class CountByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java index 675ea4d0295c5..38187617d02ac 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java @@ -15,17 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; import java.time.Duration; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator Distinct. */ public class DistinctTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java index 626ff749bd977..0f81ba834cf49 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import org.junit.Test; - /** Test operator Filter. */ public class FilterTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java index 87eb034474ddf..6b4ccd67d91f3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java @@ -15,17 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; import java.math.BigDecimal; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator FlatMap. */ public class FlatMapTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java similarity index 78% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java index 8012205cccc3b..2427e8b00516f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/HintTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java @@ -15,27 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.MockStreamDataSource; -import cz.seznam.euphoria.core.client.io.VoidSink; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.operator.hint.SizeHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.executor.FlowUnfolder; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.MockStreamDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.VoidSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.junit.Test; + import java.time.Duration; import java.util.Set; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** Test usage of hints in different operators. */ public class HintTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java index 0f483067644ea..09ffd96df793b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java @@ -15,22 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.hint.SizeHint; -import cz.seznam.euphoria.core.client.util.Pair; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; + import java.time.Duration; import java.util.Optional; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator Join. */ public class JoinTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java index 86146862a63a6..25c27e066fdac 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java @@ -15,17 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.operator.hint.SizeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; import org.junit.Test; +import static org.junit.Assert.*; + /** Test operator MapElement. */ public class MapElementsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java index 5d0af03131a3c..80429c2956eb0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java @@ -15,21 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.util.Pair; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; + import java.time.Duration; import java.util.stream.StreamSupport; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator ReduceByKey. */ public class ReduceByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java index acb2a19b7e9b6..83386c3aa4702 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java @@ -15,25 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.util.Pair; -import java.time.Duration; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; +import java.time.Duration; + +import static org.junit.Assert.*; + /** Test operator ReduceStateByKey. */ public class ReduceStateByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java index e5542525afe6e..57a77ba541bdb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java @@ -15,21 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.executor.util.SingleValueContext; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; +import org.junit.Test; + import java.time.Duration; import java.util.stream.Stream; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test behavior of operator {@code ReduceWindow}. */ public class ReduceWindowTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java index 090bb7303d322..a47d65b2e967a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java @@ -15,19 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.util.Pair; import java.time.Duration; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test behavior of operator {@code SumByKey}. */ public class SumByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java index 3faf9e63d3f0a..961117e30320c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java @@ -15,21 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.util.Triple; -import java.time.Duration; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; import org.junit.Test; +import java.time.Duration; + +import static org.junit.Assert.*; + /** Test behavior of operator {@code TopPerKey}. */ public class TopPerKeyTest { @Test diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java index a302ba77c3c63..aae48d4a7bd76 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import static org.junit.Assert.assertEquals; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.junit.Test; +import static org.junit.Assert.assertEquals; + /** Test behavior of operator {@code Union}. */ public class UnionTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java index 169cfe4adf8f6..70d78505a3298 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/operator/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.operator; +package org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; import java.util.ArrayList; import java.util.List; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java index 8d47ae173f3f5..0c6fba1ab7d05 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/client/util/IOUtilsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.client.util; +package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; +import org.junit.Test; -import cz.seznam.euphoria.core.util.IOUtils; import java.io.IOException; import java.util.Arrays; import java.util.stream.Stream; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** Test behavior of IOUtils. */ public class IOUtilsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java index d0be6194e0993..54c02c3719aa9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/FlowUnfolderTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java @@ -15,34 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.executor; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.Datasets; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.MockStreamDataSource; -import cz.seznam.euphoria.core.client.io.StdoutSink; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.SingleInputOperator; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.FlowUnfolder.InputOperator; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.MockStreamDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.StdoutSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.SingleInputOperator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder.InputOperator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; +import org.junit.Before; +import org.junit.Test; + import java.time.Duration; import java.util.Arrays; import java.util.Collection; @@ -51,8 +50,8 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import org.junit.Before; -import org.junit.Test; + +import static org.junit.Assert.*; /** {@code FlowUnfolder} test suite. */ public class FlowUnfolderTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java index eae07bbdcd4f7..3808c9e5e8239 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/graph/DAGTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.graph; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; import com.google.common.collect.Iterables; +import org.junit.Test; + import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** DAG test suite. */ public class DAGTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java index f389b52c53867..261e15c03bf7f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/FsSpillingListStorageTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java @@ -15,23 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; import com.google.common.collect.Lists; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; + +import static org.junit.Assert.*; /** TODO: complete javadoc. */ public class FsSpillingListStorageTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java index abb3765a2dbb4..8cb85e835e39f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/GenericSpillToolsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java @@ -15,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; -import cz.seznam.euphoria.core.client.io.ExternalIterable; import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.StreamSupport; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** Test suite for {@code GenericSpillTools}. */ public class GenericSpillToolsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/JavaSerializationFactory.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/JavaSerializationFactory.java index 135c92effaa52..5a69058599806 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/JavaSerializationFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/JavaSerializationFactory.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; import java.io.IOException; import java.io.ObjectInputStream; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java index eb6e8383bf738..0ad31c4b2da77 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/executor/io/TmpFolderSpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.executor.io; +package org.apache.beam.sdk.extensions.euphoria.core.executor.io; + +import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.junit.rules.TemporaryFolder; /** TODO: complete javadoc. */ class TmpFolderSpillFileFactory implements SpillFileFactory { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java index 673c2c035bcf7..e65d337b847ff 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.testing; +package org.apache.beam.sdk.extensions.euphoria.core.testing; + +import org.junit.Assert; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.junit.Assert; /** This is duplicated from {@code euphoria-testing} due to maven lifecycle cyclic dependency. */ public class DatasetAssert { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java index f3bcb4ff495db..c0dffebd736b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/time/TimeProvidersTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java @@ -15,16 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.time; +package org.apache.beam.sdk.extensions.euphoria.core.time; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.Test; import java.time.Duration; import java.util.Calendar; import java.util.Date; import java.util.TimeZone; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** TODO: complete javadoc. */ public class TimeProvidersTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java rename to sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java index ecfefc9d24ecc..7b226fe002c36 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/cz/seznam/euphoria/core/util/SettingsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.core.util; - -import static org.junit.Assert.assertEquals; +package org.apache.beam.sdk.extensions.euphoria.core.util; import org.junit.Test; +import static org.junit.Assert.assertEquals; + /** TODO: complete javadoc. */ public class SettingsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java similarity index 62% rename from sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java rename to sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java index d886570a8f33f..65f4b3819fc7d 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java @@ -15,35 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.fluent; +package org.apache.beam.sdk.extensions.euphoria.fluent; -import static java.util.Objects.requireNonNull; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Builders.Output; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.operator.Builders.Output; -import cz.seznam.euphoria.core.client.operator.Distinct; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.executor.Executor; +import static java.util.Objects.requireNonNull; /** TODO: complete javadoc. */ public class Dataset { - private final cz.seznam.euphoria.core.client.dataset.Dataset wrap; + private final org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset wrap; - Dataset(cz.seznam.euphoria.core.client.dataset.Dataset wrap) { + Dataset(org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset wrap) { this.wrap = requireNonNull(wrap); } - public cz.seznam.euphoria.core.client.dataset.Dataset unwrap() { + public org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset unwrap() { return this.wrap; } public Dataset apply( - UnaryFunction, Output> output) { + UnaryFunction, Output> output) { return new Dataset<>(requireNonNull(output.apply(this.wrap)).output()); } diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java similarity index 67% rename from sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java rename to sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java index 20015551ea7a2..73df1e8d2a543 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java @@ -15,27 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.fluent; +package org.apache.beam.sdk.extensions.euphoria.fluent; -import static java.util.Objects.requireNonNull; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.util.Settings; +import static java.util.Objects.requireNonNull; /** TODO: complete javadoc. */ public class Flow { - private final cz.seznam.euphoria.core.client.flow.Flow wrap; + private final org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow wrap; - Flow(cz.seznam.euphoria.core.client.flow.Flow wrap) { + Flow(org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow wrap) { this.wrap = requireNonNull(wrap); } public static Flow create(String name) { - return new Flow(cz.seznam.euphoria.core.client.flow.Flow.create(name)); + return new Flow(org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow.create(name)); } public static Flow create(String name, Settings settings) { - return new Flow(cz.seznam.euphoria.core.client.flow.Flow.create(name, settings)); + return new Flow(org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow.create(name, settings)); } public Dataset read(DataSource src) { diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java rename to sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java index 6845fb79d3cbd..dd0c7cc006556 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.fluent; +package org.apache.beam.sdk.extensions.euphoria.fluent; -import cz.seznam.euphoria.core.util.Settings; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; /** Helper class providing convenient start points into the fluent api. */ public class Fluent { @@ -30,7 +30,7 @@ public static Flow flow(String name, Settings settings) { return Flow.create(name, settings); } - public static Dataset lift(cz.seznam.euphoria.core.client.dataset.Dataset xs) { + public static Dataset lift(org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset xs) { return new Dataset<>(xs); } } diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/package-info.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/package-info.java rename to sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/package-info.java index 5c4a6cea3c316..6c6b3bf257bbd 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/package-info.java @@ -17,4 +17,4 @@ */ /** Provides fluent API. */ -package cz.seznam.euphoria.fluent; +package org.apache.beam.sdk.extensions.euphoria.fluent; diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java rename to sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java index ba57ae36a496f..2d861b0889889 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/cz/seznam/euphoria/fluent/FluentTest.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.fluent; +package org.apache.beam.sdk.extensions.euphoria.fluent; -import static java.util.Arrays.asList; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.executor.local.LocalExecutor; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.windowing.Count; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.executor.local.LocalExecutor; import java.time.Duration; import java.util.Set; import java.util.stream.Collectors; -import org.junit.Test; + +import static java.util.Arrays.asList; /** Test behavior of Fluent API. */ public class FluentTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java index ab3c01cee4a24..10c73129f10f8 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AbstractTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java @@ -15,12 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.ObjectInputStream; import java.util.ArrayList; @@ -31,8 +34,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Base class for various time triggering strategies. */ public abstract class AbstractTriggerScheduler implements TriggerScheduler { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java similarity index 74% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java index 9513c7036949c..7881d6a707ade 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/AttachedWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.triggers.NoopTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; import java.util.Collections; class AttachedWindowing> implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Collector.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Collector.java index fce5c4435d3ca..679d6703457dc 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Collector.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; interface Collector { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Datum.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Datum.java index 3b538de69521a..c1820420739ba 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Datum.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Datum.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; /** * Object passed inside local executor's processing pipelines. This is wrapper for * client data * diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecPath.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecPath.java index f01b34973cbd7..6d7c2263f4597 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecPath.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; /** A series of transformations with single output operator. */ class ExecPath { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java index fc19609e5af05..adc7fe3ffd4d1 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecUnit.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java index 66035f4085d39..ecb5d1951233d 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/KeyedWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; import java.util.Objects; /** diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java index ca4591701fea9..a3e4647d12f3e 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java @@ -15,39 +15,43 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.VoidAccumulatorProvider; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.ExtractEventTime; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.BoundedDataSource; -import cz.seznam.euphoria.core.client.io.BoundedReader; -import cz.seznam.euphoria.core.client.io.DataSink; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.io.UnboundedPartition; -import cz.seznam.euphoria.core.client.io.UnboundedReader; -import cz.seznam.euphoria.core.client.io.Writer; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.executor.FlowUnfolder; -import cz.seznam.euphoria.core.executor.FlowUnfolder.InputOperator; -import cz.seznam.euphoria.core.executor.VectorClock; -import cz.seznam.euphoria.core.executor.graph.DAG; -import cz.seznam.euphoria.core.executor.graph.Node; -import cz.seznam.euphoria.core.util.Settings; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedReader; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedPartition; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedReader; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Writer; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder; +import org.apache.beam.sdk.extensions.euphoria.core.executor.FlowUnfolder.InputOperator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.VectorClock; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -73,9 +77,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.annotation.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Local executor for testing and not fault tolerant local applications. */ public class LocalExecutor implements Executor { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java index 719e809f112dc..b0abca9b36148 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.io.ExternalIterable; -import cz.seznam.euphoria.core.client.io.SpillTools; -import cz.seznam.euphoria.core.executor.util.InMemExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.executor.util.InMemExternalIterable; + import java.util.ArrayList; import java.util.Collection; import java.util.Collections; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStateContext.java similarity index 79% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStateContext.java index b4164e9789adc..d3eba744cfb4b 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStateContext.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStateContext.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.io.SpillTools; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; /** Implementation of {@code StateContext} for {@code LocalExecutor}. */ class LocalStateContext implements StateContext { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java index 7c171d18b8777..1ae8f3e27463a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/LocalStorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; import java.util.ArrayList; import java.util.List; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/NoopTriggerScheduler.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/NoopTriggerScheduler.java index a1fa47827e1f0..5bf8faa1543c3 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/NoopTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/NoopTriggerScheduler.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * TODO: add comment. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ProcessingTimeTriggerScheduler.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ProcessingTimeTriggerScheduler.java index 82cbf8de11b6d..a2cc80a284c7c 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ProcessingTimeTriggerScheduler.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; /** Trigger scheduler based on real wall-clock time (processing time). */ public class ProcessingTimeTriggerScheduler extends AbstractTriggerScheduler { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java index 9fdc9c3609e0e..b976f8ce76293 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ReduceStateByKeyReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java @@ -15,36 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; - -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toSet; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.MergingStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateFactory; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateMerger; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.Storage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.MergingStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StateFactory; -import cz.seznam.euphoria.core.client.operator.state.StateMerger; -import cz.seznam.euphoria.core.client.operator.state.Storage; -import cz.seznam.euphoria.core.client.operator.state.StorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.StorageProvider; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.util.Settings; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -56,8 +55,10 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; class ReduceStateByKeyReducer implements Runnable { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/TriggerScheduler.java similarity index 94% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/TriggerScheduler.java index 325b49f3a94e2..44d6b4993d25e 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/TriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/TriggerScheduler.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * Schedules and fires registered triggers according to internal time. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Triggerable.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Triggerable.java index 2ebd677ebed33..704c085f21922 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Triggerable.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Triggerable.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * TODO: add javadoc. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java index 1f821d323ba0f..666071e7f2f13 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkEmitStrategy.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java index 23db8e95b9065..c8ab1be2259b1 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WatermarkTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; + import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java similarity index 73% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java index ba1c4deee62da..5c1937a289555 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/WindowedElementCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.Context; -import cz.seznam.euphoria.core.util.Settings; import java.util.Objects; import java.util.function.Supplier; class WindowedElementCollector implements Context, Collector { - private final cz.seznam.euphoria.executor.local.Collector wrap; + private final org.apache.beam.sdk.extensions.euphoria.executor.local.Collector wrap; private final Supplier stampSupplier; private final AccumulatorProvider.Factory accumulatorFactory; private final Settings settings; @@ -37,7 +38,7 @@ class WindowedElementCollector implements Context, Collector { private AccumulatorProvider accumulators; WindowedElementCollector( - cz.seznam.euphoria.executor.local.Collector wrap, + org.apache.beam.sdk.extensions.euphoria.executor.local.Collector wrap, Supplier stampSupplier, AccumulatorProvider.Factory accumulatorFactory, Settings settings) { diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/package-info.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/package-info.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/package-info.java rename to sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/package-info.java index 36f2e7cbb39f9..2a7329431d452 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/package-info.java @@ -19,4 +19,4 @@ /** * TODO add package comment. */ -package cz.seznam.euphoria.testing; +package org.apache.beam.sdk.extensions.euphoria.executor.local; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java index 56a22cc7027aa..537e3f5dc5a7a 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/BasicOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java @@ -15,33 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; - -import static java.util.Arrays.asList; -import static java.util.stream.Collectors.toList; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Session; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.Distinct; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.testing.DatasetAssert; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; +import org.junit.Test; + import java.time.Duration; import java.util.HashSet; import java.util.List; @@ -50,7 +48,10 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.junit.Test; + +import static java.util.Arrays.asList; +import static java.util.stream.Collectors.toList; +import static org.junit.Assert.assertTrue; /** Test basic operator functionality and ability to compile. */ public class BasicOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java index 29add65591cd4..fee292b68e33e 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/JoinOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java @@ -15,30 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; - -import static java.util.Arrays.asList; +package org.apache.beam.sdk.extensions.euphoria.executor.local; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; +import org.junit.Before; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.Filter; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.FullJoin; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.testing.DatasetAssert; import java.time.Duration; import java.util.List; -import org.junit.Before; -import org.junit.Test; + +import static java.util.Arrays.asList; /** * Test of Join operator. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java index 37fe0a81d9bb1..5ebcf453c9c15 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/LocalExecutorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java @@ -15,48 +15,46 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.testing.DatasetAssert; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -70,9 +68,8 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; + +import static org.junit.Assert.*; /** * {@code LocalExecutor} test suite. The {@code LocalExecutor} stands on the basic operators, so we diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Util.java similarity index 95% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Util.java index ae9c46bcd380a..c19b91a50eb5c 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Util.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import java.util.ArrayList; import java.util.Collection; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java index 3d536c1ec632a..793effa077839 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/VectorClockTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.executor.local; -import static org.junit.Assert.assertEquals; - -import cz.seznam.euphoria.core.executor.VectorClock; +import org.apache.beam.sdk.extensions.euphoria.core.executor.VectorClock; import org.junit.Test; +import static org.junit.Assert.assertEquals; + /** Test vector clocks. */ public class VectorClockTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java index 1f969dc2e002a..edfccfcdcefca 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java @@ -15,39 +15,40 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Count; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.util.Fold; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.testing.DatasetAssert; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Fold; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; +import org.junit.Before; +import org.junit.Test; + import java.time.Duration; import java.util.Comparator; import java.util.List; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import org.junit.Before; -import org.junit.Test; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * Collection of windowing teats. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalExecutorProvider.java similarity index 74% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalExecutorProvider.java index 346ce2283370c..4ea6a7fda15fd 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalExecutorProvider.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local.testkit; +package org.apache.beam.sdk.extensions.euphoria.executor.local.testkit; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.executor.local.LocalExecutor; -import cz.seznam.euphoria.executor.local.WatermarkTriggerScheduler; -import cz.seznam.euphoria.operator.test.junit.ExecutorEnvironment; -import cz.seznam.euphoria.operator.test.junit.ExecutorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.executor.local.LocalExecutor; +import org.apache.beam.sdk.extensions.euphoria.executor.local.WatermarkTriggerScheduler; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorEnvironment; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; /** * Executor provider used for testing. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalOperatorTest.java similarity index 86% rename from sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java rename to sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalOperatorTest.java index 73cc9d659d911..ce3c7a0dbb224 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalOperatorTest.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.executor.local.testkit; +package org.apache.beam.sdk.extensions.euphoria.executor.local.testkit; -import cz.seznam.euphoria.operator.test.AllOperatorsSuite; +import org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite; /** * Local operator test suite. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/AllOperatorsSuite.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/AllOperatorsSuite.java index 624cd05885117..65d6f7c65da47 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/AllOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/AllOperatorsSuite.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import cz.seznam.euphoria.operator.test.junit.ExecutorProvider; -import cz.seznam.euphoria.operator.test.junit.ExecutorProviderRunner; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProviderRunner; import org.junit.runner.RunWith; import org.junit.runners.Suite; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java index 2784b3a4107fa..50053af0163b1 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/BroadcastHashJoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.LeftJoin; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.RightJoin; -import cz.seznam.euphoria.core.client.operator.hint.SizeHint; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; + import java.util.Arrays; import java.util.List; import java.util.Optional; -import org.junit.Test; /** * Collection of broadcast hash join tests. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java index 970dba3d6e1b0..4246448175657 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java @@ -15,20 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.CountByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; -import cz.seznam.euphoria.operator.test.junit.Processing.Type; import java.time.Duration; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test operator {@code CountByKey}. */ @Processing(Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java similarity index 84% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java index c95a21e39ab67..849cf1d43c29f 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.Distinct; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; -import cz.seznam.euphoria.operator.test.junit.Processing.Type; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; +import org.junit.Test; + import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test for the {@link Distinct} operator. */ @Processing(Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java index ca915e981c579..d05d5bbb9c3d6 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.operator.Filter; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test operator {@code Filter}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java similarity index 83% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java index dc144a9718569..a312a07fe18dd 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import static org.junit.Assert.assertEquals; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; /** Test operator {@code FlatMap}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/IntWindow.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/IntWindow.java index 97cd3addecc85..f4dcee0669714 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/IntWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/IntWindow.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * Test-purpose {@link Window} identified by integer. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java index c3cd381cba949..960c67464b2bf 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java @@ -15,40 +15,41 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import static org.junit.Assert.assertEquals; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Either; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Session; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FullJoin; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.LeftJoin; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.RightJoin; -import cz.seznam.euphoria.core.client.triggers.NoopTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.util.Either; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; /** Test operator {@code Join}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java index 2fccde64c06be..f6c111ceed61e 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/JoinWindowEnforcementTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java @@ -15,28 +15,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.WindowingRequiredException; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runners.Parameterized; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Count; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.WindowingRequiredException; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized; /** * A collection of test methods. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java index 08d59066b7464..ea880845fb891 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java @@ -15,21 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import static org.junit.Assert.assertEquals; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctionEnv; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.functional.UnaryFunctionEnv; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; /** Tests for operator {@code MapElements}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index 9bb6b2c2978c5..75943d2edde34 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -15,47 +15,45 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.operator.test; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Count; -import cz.seznam.euphoria.core.client.dataset.windowing.GlobalWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Session; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.CountTrigger; -import cz.seznam.euphoria.core.client.triggers.NoopTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.type.TypeHint; -import cz.seznam.euphoria.core.client.util.Fold; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Fold; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; + import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -68,7 +66,8 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.junit.Test; + +import static org.junit.Assert.*; /** Test operator {@code ReduceByKey}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java index b76921b738b2c..42a49f4f7d66e 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java @@ -15,41 +15,40 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +package org.apache.beam.sdk.extensions.euphoria.operator.test; import com.google.common.collect.Lists; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Count; -import cz.seznam.euphoria.core.client.dataset.windowing.Session; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeSliding; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.functional.UnaryFunctor; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.state.ListStorage; -import cz.seznam.euphoria.core.client.operator.state.ListStorageDescriptor; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.StateFactory; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.CountTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeSliding; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateFactory; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; + import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -58,7 +57,9 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** Test operator {@code ReduceStateByKey}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java similarity index 81% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java index 0ce17ba8604ca..e901a25e759b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java @@ -15,19 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test operator {@code ReduceByKey}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java index 462fc4591e84f..581e2d20026b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java @@ -15,27 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.CountByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.CountByKey; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; /** Test that a sub-flow applied on sink is correctly preserved. */ public class SinkTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java similarity index 76% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java index c98ca14f92c67..b9975696629d3 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.SumByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.operator.SumByKey; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test operator {@code SumByKey}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java index e1f2d04e20683..6a2b111377a48 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java @@ -15,19 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; -import static java.util.Arrays.asList; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.TopPerKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.operator.TopPerKey; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.io.Serializable; import java.util.List; import java.util.Objects; -import org.junit.Test; + +import static java.util.Arrays.asList; /** * TODO: add javadoc. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java index 301bf695c3351..156148dffcf15 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.Union; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; + import java.util.Arrays; import java.util.List; -import org.junit.Test; /** Test for operator {@code Union}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java similarity index 79% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java index 1978214dfea8d..1079ca388f4ca 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.util.Pair; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java similarity index 77% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java index 90701f06558f1..2212bf5e049b5 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WatermarkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java @@ -15,23 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.util.Arrays; import java.util.List; -import org.junit.Test; /** * TODO: add javadoc. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java index edfb551d73bea..2f13ce9b205eb 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java @@ -15,38 +15,37 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +package org.apache.beam.sdk.extensions.euphoria.operator.test; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Distinct; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceWindow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.State; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.junit.Test; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.dataset.windowing.MergingWindowing; -import cz.seznam.euphoria.core.client.dataset.windowing.Session; -import cz.seznam.euphoria.core.client.dataset.windowing.Time; -import cz.seznam.euphoria.core.client.dataset.windowing.TimeInterval; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.io.Collector; -import cz.seznam.euphoria.core.client.operator.AssignEventTime; -import cz.seznam.euphoria.core.client.operator.Distinct; -import cz.seznam.euphoria.core.client.operator.FlatMap; -import cz.seznam.euphoria.core.client.operator.MapElements; -import cz.seznam.euphoria.core.client.operator.ReduceByKey; -import cz.seznam.euphoria.core.client.operator.ReduceStateByKey; -import cz.seznam.euphoria.core.client.operator.ReduceWindow; -import cz.seznam.euphoria.core.client.operator.state.State; -import cz.seznam.euphoria.core.client.operator.state.StateContext; -import cz.seznam.euphoria.core.client.operator.state.ValueStorage; -import cz.seznam.euphoria.core.client.operator.state.ValueStorageDescriptor; -import cz.seznam.euphoria.core.client.triggers.TimeTrigger; -import cz.seznam.euphoria.core.client.triggers.Trigger; -import cz.seznam.euphoria.core.client.triggers.TriggerContext; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.core.client.util.Sums; -import cz.seznam.euphoria.core.client.util.Triple; -import cz.seznam.euphoria.operator.test.junit.AbstractOperatorTest; -import cz.seznam.euphoria.operator.test.junit.Processing; import java.time.Duration; import java.time.Instant; import java.util.Arrays; @@ -54,7 +53,9 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** Tests capabilities of {@link Windowing}. */ @Processing(Processing.Type.ALL) diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java index 7c0bbdb20de91..2014c756811ae 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongCounter.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Counter; import java.util.concurrent.atomic.AtomicLong; final class LongCounter diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java index a2e52ecb3b3f9..e914c9743f3f1 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/LongHistogram.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Histogram; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java index fbb12ad727620..a507946453a16 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/NanosecondTimer.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; import com.google.common.collect.Maps; -import cz.seznam.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; + import java.time.Duration; import java.util.Map; import java.util.concurrent.TimeUnit; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java index b95e15f7a56d8..55170e7572817 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; - -import cz.seznam.euphoria.core.client.accumulators.Accumulator; -import cz.seznam.euphoria.core.client.accumulators.AccumulatorProvider; -import cz.seznam.euphoria.core.client.accumulators.Counter; -import cz.seznam.euphoria.core.client.accumulators.Histogram; -import cz.seznam.euphoria.core.client.accumulators.Timer; -import cz.seznam.euphoria.core.util.Settings; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Accumulator; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; + import java.io.ObjectStreamException; import java.time.Duration; import java.util.HashMap; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SnapshotProvider.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SnapshotProvider.java index 8ad51e1af1137..ce10795cd129d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SnapshotProvider.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; import java.time.Duration; import java.util.Map; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/Snapshotable.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/Snapshotable.java index 7b11e9ca1ccb4..60e3db3a4b687 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/Snapshotable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; interface Snapshotable { V getSnapshot(); diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/package-info.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/package-info.java similarity index 91% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/package-info.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/package-info.java index e68f306634928..03b178aefc17a 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/package-info.java @@ -19,4 +19,4 @@ /** * Accumulators to be used in operator test suite. */ -package cz.seznam.euphoria.operator.test.accumulators; +package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java similarity index 89% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java index 3d165743f5180..7c47ef9244855 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/AbstractOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java @@ -15,21 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators + .SingleJvmAccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; -import static com.google.common.base.Preconditions.checkNotNull; -import static org.junit.Assert.assertEquals; - -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.util.Settings; -import cz.seznam.euphoria.operator.test.accumulators.SingleJvmAccumulatorProvider; -import cz.seznam.euphoria.operator.test.accumulators.SnapshotProvider; -import cz.seznam.euphoria.operator.test.junit.Processing.Type; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; @@ -37,6 +36,9 @@ import java.util.Map; import java.util.concurrent.ExecutionException; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; + /** * Base class for test description of a test case. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorEnvironment.java similarity index 87% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorEnvironment.java index 8bbc6e860ef69..33fc3bb2471e0 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorEnvironment.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; -import cz.seznam.euphoria.core.executor.Executor; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; /** * Source of {@link Executor} with ability to shutdown it. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProvider.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProvider.java index 64890c3e0915d..510ba81d3ee04 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProvider.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; /** * Provider of {@lin ExecutorEnvironment}. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java similarity index 98% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java index 9c5aceaab3fd5..3ca7c217595f0 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProviderRunner.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java @@ -15,19 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; -import static com.google.common.base.Preconditions.checkArgument; - -import cz.seznam.euphoria.operator.test.junit.Processing.Type; -import java.lang.reflect.AnnotatedElement; -import java.lang.reflect.Modifier; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; import org.junit.Ignore; import org.junit.Test; import org.junit.runner.Description; @@ -43,6 +33,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.AnnotatedElement; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Stream; + +import static com.google.common.base.Preconditions.checkArgument; + /** * TODO: add javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java index 664ed89ec8ac8..787104611d4c6 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/Processing.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; import com.google.common.collect.Lists; + import java.lang.annotation.Inherited; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/package-info.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/package-info.java similarity index 92% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/package-info.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/package-info.java index 58f0bb5dc0923..ca6d76ba38892 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/package-info.java @@ -19,4 +19,4 @@ /** * Collection of tst utility classes. */ -package cz.seznam.euphoria.operator.test.junit; +package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/package-info.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/package-info.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/package-info.java rename to sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/package-info.java index b433894f507b3..9692a161aa5f7 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/package-info.java @@ -18,6 +18,6 @@ /** * Euphoria operators test suite, see - * {@link cz.seznam.euphoria.operator.test.AllOperatorsSuite}. + * {@link org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite}. */ -package cz.seznam.euphoria.operator.test; +package org.apache.beam.sdk.extensions.euphoria.operator.test; diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java rename to sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java index 09f77775512e4..3697ed8a380f9 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/AbstractFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.testing; +package org.apache.beam.sdk.extensions.euphoria.testing; + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; -import cz.seznam.euphoria.core.client.dataset.Dataset; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.executor.Executor; import java.util.List; /** diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java similarity index 97% rename from sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java rename to sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java index 1c1c8c4b7c44d..2e0b7abf2e65d 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/cz/seznam/euphoria/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.testing; +package org.apache.beam.sdk.extensions.euphoria.testing; + +import org.junit.Assert; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.junit.Assert; /** * Test wide utility methods for dataset assertion. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/package-info.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/package-info.java rename to sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java index bd73ef4ce9b9c..da136365b4057 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/package-info.java @@ -19,4 +19,4 @@ /** * TODO add package comment. */ -package cz.seznam.euphoria.executor.local; +package org.apache.beam.sdk.extensions.euphoria.testing; From 4ad6de7ac32818c2305646f9b27a5bdd76ee9955 Mon Sep 17 00:00:00 2001 From: David Moravek Date: Tue, 15 May 2018 22:14:02 +0200 Subject: [PATCH 3/3] [BEAM-3902] Move euphoria to apache namespace. --- .../euphoria/beam/BeamExecutorContext.java | 15 +++--- .../extensions/euphoria/beam/BeamFlow.java | 13 +++-- .../euphoria/beam/DoFnCollector.java | 7 ++- .../euphoria/beam/FlatMapTranslator.java | 3 +- .../euphoria/beam/FlowTranslator.java | 5 +- .../euphoria/beam/InputTranslator.java | 3 +- .../beam/LazyAccumulatorProvider.java | 5 +- .../euphoria/beam/ReduceByKeyTranslator.java | 3 +- .../euphoria/beam/SingleValueCollector.java | 3 +- .../beam/WrappedPCollectionOperator.java | 5 +- .../euphoria/beam/coder/PairCoder.java | 11 ++-- .../euphoria/beam/io/BeamBoundedSource.java | 12 ++--- .../euphoria/beam/io/BeamUnboundedSource.java | 11 ++-- .../euphoria/beam/io/BeamWriteSink.java | 5 +- .../euphoria/beam/io/KryoCoder.java | 7 ++- .../euphoria/beam/window/BeamWindowFn.java | 9 ++-- .../beam/window/BeamWindowedElement.java | 3 +- .../euphoria/beam/BeamFlowTest.java | 9 ++-- .../extensions/euphoria/beam/FlatMapTest.java | 5 +- .../euphoria/beam/ReduceByKeyTest.java | 11 ++-- .../beam/testkit/BeamExecutorProvider.java | 3 +- .../beam/testkit/BeamOperatorsSuite.java | 7 +-- .../accumulators/AccumulatorProvider.java | 3 +- .../core/client/accumulators/Timer.java | 3 +- .../accumulators/VoidAccumulatorProvider.java | 5 +- .../euphoria/core/client/dataset/Dataset.java | 7 ++- .../core/client/dataset/InputDataset.java | 5 +- .../core/client/dataset/OutputDataset.java | 5 +- .../core/client/dataset/windowing/Count.java | 4 +- .../dataset/windowing/GlobalWindowing.java | 10 ++-- .../dataset/windowing/MergingWindowing.java | 3 +- .../client/dataset/windowing/Session.java | 19 ++++--- .../core/client/dataset/windowing/Time.java | 15 +++--- .../client/dataset/windowing/TimeSliding.java | 11 ++-- .../core/client/dataset/windowing/Window.java | 3 +- .../client/dataset/windowing/Windowing.java | 3 +- .../euphoria/core/client/flow/Flow.java | 23 ++++---- .../client/functional/BinaryFunction.java | 3 +- .../core/client/functional/BinaryFunctor.java | 3 +- .../core/client/functional/Consumer.java | 3 +- .../client/functional/ExtractEventTime.java | 3 +- .../client/functional/ReduceFunction.java | 3 +- .../core/client/functional/ReduceFunctor.java | 3 +- .../client/functional/TernaryFunction.java | 3 +- .../core/client/functional/TypeHintAware.java | 3 +- .../core/client/functional/UnaryFunction.java | 3 +- .../client/functional/UnaryFunctionEnv.java | 3 +- .../core/client/functional/UnaryFunctor.java | 3 +- .../core/client/functional/VoidFunction.java | 3 +- .../core/client/io/BoundedDataSource.java | 3 +- .../core/client/io/BoundedReader.java | 3 +- .../core/client/io/CloseableIterator.java | 3 +- .../euphoria/core/client/io/DataSink.java | 5 +- .../euphoria/core/client/io/DataSinks.java | 3 +- .../euphoria/core/client/io/DataSource.java | 3 +- .../core/client/io/ExternalIterable.java | 3 +- .../euphoria/core/client/io/ListDataSink.java | 9 ++-- .../core/client/io/ListDataSource.java | 3 +- .../core/client/io/MultiDataSink.java | 5 +- .../euphoria/core/client/io/SpillTools.java | 5 +- .../euphoria/core/client/io/StdoutSink.java | 3 +- .../core/client/io/UnboundedDataSource.java | 3 +- .../core/client/io/UnboundedPartition.java | 3 +- .../client/io/UnsplittableBoundedSource.java | 3 +- .../euphoria/core/client/io/VoidSink.java | 3 +- .../euphoria/core/client/io/Writer.java | 3 +- .../euphoria/core/client/lib/Split.java | 5 +- .../core/client/operator/AssignEventTime.java | 5 +- .../core/client/operator/CountByKey.java | 7 ++- .../core/client/operator/Distinct.java | 9 ++-- .../client/operator/ElementWiseOperator.java | 3 +- .../euphoria/core/client/operator/Filter.java | 5 +- .../core/client/operator/FlatMap.java | 9 ++-- .../core/client/operator/FullJoin.java | 5 +- .../euphoria/core/client/operator/Join.java | 11 ++-- .../core/client/operator/LeftJoin.java | 5 +- .../core/client/operator/MapElements.java | 7 ++- .../core/client/operator/Operator.java | 11 ++-- .../operator/OptionalMethodBuilder.java | 3 +- .../core/client/operator/ReduceByKey.java | 13 +++-- .../client/operator/ReduceStateByKey.java | 12 ++--- .../core/client/operator/ReduceWindow.java | 7 ++- .../core/client/operator/RightJoin.java | 5 +- .../client/operator/SingleInputOperator.java | 5 +- .../StateAwareWindowWiseOperator.java | 3 +- ...ateAwareWindowWiseSingleInputOperator.java | 9 ++-- .../core/client/operator/SumByKey.java | 9 ++-- .../core/client/operator/TopPerKey.java | 11 ++-- .../euphoria/core/client/operator/Union.java | 17 +++--- .../client/operator/WindowWiseOperator.java | 3 +- .../core/client/operator/hint/OutputHint.java | 3 +- .../client/operator/state/StateContext.java | 3 +- .../client/operator/state/StateFactory.java | 5 +- .../client/operator/state/StateMerger.java | 9 ++-- .../operator/state/StorageDescriptor.java | 3 +- .../operator/state/StorageProvider.java | 3 +- .../triggers/AfterFirstCompositeTrigger.java | 3 +- .../core/client/triggers/Trigger.java | 3 +- .../core/client/type/AbstractTypeAware.java | 3 +- .../client/type/TypeAwareReduceFunctor.java | 3 +- .../euphoria/core/client/type/TypeHint.java | 3 +- .../euphoria/core/client/util/Either.java | 5 +- .../euphoria/core/client/util/Fold.java | 5 +- .../euphoria/core/client/util/Max.java | 3 +- .../euphoria/core/client/util/Pair.java | 3 +- .../euphoria/core/client/util/Sums.java | 3 +- .../euphoria/core/client/util/Triple.java | 3 +- .../core/executor/AbstractExecutor.java | 7 ++- .../euphoria/core/executor/Executor.java | 5 +- .../euphoria/core/executor/FlowUnfolder.java | 17 +++--- .../euphoria/core/executor/FlowValidator.java | 13 +++-- .../euphoria/core/executor/VectorClock.java | 3 +- .../euphoria/core/executor/graph/DAG.java | 5 +- .../euphoria/core/executor/graph/Node.java | 5 +- .../core/executor/greduce/GroupReducer.java | 18 +++---- .../core/executor/greduce/TimerSupport.java | 7 ++- .../core/executor/greduce/TriggerStorage.java | 5 +- .../executor/io/FsSpillingListStorage.java | 7 ++- .../core/executor/io/GenericSpillTools.java | 15 +++--- .../core/executor/io/SerializerFactory.java | 3 +- .../core/executor/io/SpillFileFactory.java | 3 +- .../executor/util/InMemExternalIterable.java | 3 +- .../core/executor/util/MultiValueContext.java | 9 ++-- .../executor/util/OperatorTranslator.java | 3 +- .../executor/util/SingleValueContext.java | 3 +- .../euphoria/core/time/Scheduler.java | 3 +- .../euphoria/core/time/TimeProvider.java | 3 +- .../euphoria/core/time/TimeProviders.java | 3 +- .../euphoria/core/time/TimerScheduler.java | 7 ++- .../euphoria/core/util/InstanceUtils.java | 3 +- .../euphoria/core/util/Settings.java | 9 ++-- .../client/dataset/windowing/SessionTest.java | 11 ++-- .../dataset/windowing/TimeSlidingTest.java | 7 ++- .../client/dataset/windowing/TimeTest.java | 7 ++- .../dataset/windowing/WindowingTest.java | 13 ++--- .../euphoria/core/client/flow/TestFlow.java | 15 +++--- .../core/client/io/ListDataSinkTest.java | 4 +- .../core/client/io/MultiDataSinkTest.java | 9 ++-- .../euphoria/core/client/lib/SplitTest.java | 11 ++-- .../core/client/operator/CountByKeyTest.java | 10 ++-- .../core/client/operator/DistinctTest.java | 9 ++-- .../core/client/operator/FilterTest.java | 6 +-- .../core/client/operator/FlatMapTest.java | 9 ++-- .../core/client/operator/HintTest.java | 11 ++-- .../core/client/operator/JoinTest.java | 12 +++-- .../core/client/operator/MapElementsTest.java | 6 ++- .../core/client/operator/ReduceByKeyTest.java | 13 +++-- .../client/operator/ReduceStateByKeyTest.java | 10 ++-- .../client/operator/ReduceWindowTest.java | 11 ++-- .../core/client/operator/SumByKeyTest.java | 10 ++-- .../core/client/operator/TopPerKeyTest.java | 10 ++-- .../core/client/operator/UnionTest.java | 4 +- .../euphoria/core/client/operator/Util.java | 5 +- .../core/client/util/IOUtilsTest.java | 9 ++-- .../core/executor/FlowUnfolderTest.java | 23 ++++---- .../euphoria/core/executor/graph/DAGTest.java | 9 ++-- .../io/FsSpillingListStorageTest.java | 15 +++--- .../executor/io/GenericSpillToolsTest.java | 15 +++--- .../io/TmpFolderSpillFileFactory.java | 3 +- .../euphoria/core/testing/DatasetAssert.java | 3 +- .../euphoria/core/time/TimeProvidersTest.java | 7 ++- .../euphoria/core/util/SettingsTest.java | 4 +- .../extensions/euphoria/fluent/Dataset.java | 8 +-- .../sdk/extensions/euphoria/fluent/Flow.java | 7 +-- .../extensions/euphoria/fluent/Fluent.java | 3 +- .../euphoria/fluent/FluentTest.java | 11 ++-- .../local/AbstractTriggerScheduler.java | 5 +- .../executor/local/AttachedWindowing.java | 3 +- .../euphoria/executor/local/ExecUnit.java | 9 ++-- .../euphoria/executor/local/KeyedWindow.java | 3 +- .../executor/local/LocalExecutor.java | 53 +++++++++---------- .../executor/local/LocalSpillTools.java | 7 ++- .../executor/local/LocalStorageProvider.java | 5 +- .../local/ReduceStateByKeyReducer.java | 31 ++++++----- .../executor/local/WatermarkEmitStrategy.java | 1 - .../local/WatermarkTriggerScheduler.java | 5 +- .../local/WindowedElementCollector.java | 5 +- .../executor/local/BasicOperatorTest.java | 25 +++++---- .../executor/local/JoinOperatorTest.java | 9 ++-- .../executor/local/LocalExecutorTest.java | 35 ++++++------ .../executor/local/VectorClockTest.java | 4 +- .../executor/local/WindowingTest.java | 21 ++++---- .../operator/test/BroadcastHashJoinTest.java | 7 ++- .../operator/test/CountByKeyTest.java | 7 ++- .../euphoria/operator/test/DistinctTest.java | 9 ++-- .../euphoria/operator/test/FilterTest.java | 5 +- .../euphoria/operator/test/FlatMapTest.java | 11 ++-- .../euphoria/operator/test/JoinTest.java | 17 +++--- .../test/JoinWindowEnforcementTest.java | 11 ++-- .../operator/test/MapElementsTest.java | 11 ++-- .../operator/test/ReduceByKeyTest.java | 31 +++++------ .../operator/test/ReduceStateByKeyTest.java | 23 ++++---- .../operator/test/ReduceWindowTest.java | 7 ++- .../euphoria/operator/test/SinkTest.java | 19 ++++--- .../euphoria/operator/test/SumByKeyTest.java | 7 ++- .../euphoria/operator/test/TopPerKeyTest.java | 11 ++-- .../euphoria/operator/test/UnionTest.java | 5 +- .../euphoria/operator/test/Util.java | 9 ++-- .../euphoria/operator/test/WatermarkTest.java | 7 ++- .../euphoria/operator/test/WindowingTest.java | 21 ++++---- .../test/accumulators/LongCounter.java | 3 +- .../test/accumulators/LongHistogram.java | 3 +- .../test/accumulators/NanosecondTimer.java | 3 +- .../SingleJvmAccumulatorProvider.java | 13 +++-- .../test/junit/AbstractOperatorTest.java | 19 ++++--- .../test/junit/ExecutorProviderRunner.java | 21 ++++---- .../operator/test/junit/Processing.java | 1 - .../euphoria/testing/AbstractFlowTest.java | 3 +- .../euphoria/testing/DatasetAssert.java | 3 +- 209 files changed, 728 insertions(+), 874 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java index e4df8677c649a..4f180a2672840 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutorContext.java @@ -17,7 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import static java.util.stream.Collectors.toList; + import com.google.common.collect.Iterables; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -44,14 +51,6 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; - -import static java.util.stream.Collectors.toList; - /** * Keeps track of mapping between Euphoria {@link Dataset} and {@link PCollection}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java index 9829371b7914e..ff5975dae7d6d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlow.java @@ -20,6 +20,12 @@ import com.google.common.collect.Iterables; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamWriteSink; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; @@ -33,13 +39,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.PCollection; -import java.time.Duration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.stream.Collectors; - /** * A {@link Flow} that can be used in Euphoria operator constructions and integrates seamlessly with * Beam. diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java index 60d830cde6517..ca902ab0af2b2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/DoFnCollector.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.io.Serializable; +import java.util.Objects; +import javax.annotation.concurrent.NotThreadSafe; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; @@ -28,10 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; import org.apache.beam.sdk.transforms.DoFn; -import javax.annotation.concurrent.NotThreadSafe; -import java.io.Serializable; -import java.util.Objects; - /** * Collector that outputs elements to {@link BeamCollector}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java index f862f058fe12e..413a5737b279d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; @@ -26,8 +27,6 @@ import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; -import javax.annotation.Nullable; - class FlatMapTranslator implements OperatorTranslator { private static PCollection doTranslate( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index 34752cfcf7c1c..e03a65e424a89 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.IdentityHashMap; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamWriteSink; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; @@ -36,9 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; -import java.util.IdentityHashMap; -import java.util.Map; - /** * This class converts Euphoria's {@code Flow} into Beam's Pipeline. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java index 74ae82410f603..67d04159cded3 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/InputTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamBoundedSource; import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamUnboundedSource; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.values.PCollection; -import java.util.Objects; - class InputTranslator implements OperatorTranslator { private static PCollection doTranslate( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java index 1ba278e5002bb..f42ca2e3ffee5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/LazyAccumulatorProvider.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.io.Serializable; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import java.io.Serializable; -import java.util.Objects; - /** * Instantiate accumulator provider on the first usage. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index 3e76171e4b6ac..4104044038f21 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -38,8 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.util.stream.StreamSupport; - /** * Translator for {@code ReduceByKey} operator. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java index 3e13e21e2f8b9..7de542a532e47 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/SingleValueCollector.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import java.io.Serializable; - /** * {@code Collector} for combinable functors. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java index 0f10968ea5eb6..bf65c340fb2bc 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/WrappedPCollectionOperator.java @@ -18,14 +18,13 @@ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; import org.apache.beam.sdk.values.PCollection; -import java.util.Collection; -import java.util.Collections; - /** * {@link Operator} that serves as a wrapper between a {@link PCollection} and {@link Dataset}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java index 53baa1cb23208..2a79c04822f3d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/coder/PairCoder.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.coder; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StructuredCoder; @@ -25,12 +30,6 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeParameter; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; - /** * Beam {@link StructuredCoder} for euphoria {@link Pair}. * @param diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java index 274800d994c4a..4abca49b3ce5d 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamBoundedSource.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.io; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedDataSource; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedDataSource; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; /** * A {@link BoundedSource} created from {@link BoundedDataSource}. @@ -60,7 +59,8 @@ public long getEstimatedSizeBytes(PipelineOptions po) throws Exception { @Override public BoundedReader createReader(PipelineOptions po) throws IOException { - final org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedReader reader = wrap.openReader(); + final org.apache.beam.sdk.extensions.euphoria.core.client.io.BoundedReader reader = + wrap.openReader(); return new BoundedReader() { private T current = null; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java index 6e88d96d49a21..e10548aa163b0 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamUnboundedSource.java @@ -17,12 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.io; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedDataSource; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.joda.time.Instant; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -30,6 +24,11 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.UnboundedDataSource; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; /** * A {@link UnboundedSource} created from {@link UnboundedDataSource}. diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java index 630af91c93638..5c94206009ea7 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/BeamWriteSink.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.io; +import java.io.IOException; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Writer; import org.apache.beam.sdk.transforms.DoFn; @@ -26,9 +28,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import java.io.IOException; -import java.util.Objects; - /** * Write to output sink using beam. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java index 33a2bf148d496..e2679109c16b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/io/KryoCoder.java @@ -20,10 +20,6 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.VoidFunction; -import org.objenesis.strategy.StdInstantiatorStrategy; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -34,6 +30,9 @@ import java.io.ObjectOutputStream; import java.io.OutputStream; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.VoidFunction; +import org.objenesis.strategy.StdInstantiatorStrategy; /** * Coder using Kryo as (de)serialization mechanism. TODO: we should drop this class entirely diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java index d895646034e6c..7d409afdbede7 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowFn.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.window; +import java.util.Collection; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; @@ -28,11 +32,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import java.util.Collection; -import java.util.Objects; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - /** * A {@code WindowFn} wrapper of {@code Windowing}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java index ca4637c3269a4..4f2036371263a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowedElement.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.window; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; -import java.util.Objects; - /** * A {@code WindowedElement} created from Beam's element. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java index cd2ed5ecb30af..3d4220c988c2f 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/BeamFlowTest.java @@ -19,6 +19,10 @@ package org.apache.beam.sdk.extensions.euphoria.beam; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.Serializable; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -44,11 +48,6 @@ import org.junit.Ignore; import org.junit.Test; -import java.io.Serializable; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - /** * Test for {@link BeamFlow}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java index 7e30be17196c6..26164e3607440 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/FlatMapTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.Arrays; +import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; @@ -26,9 +28,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.junit.Test; -import java.util.Arrays; -import java.util.concurrent.ExecutionException; - /** * Test {@code FlatMap} operator's integration with beam. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index d7835a4897ec1..2a913ab9746a5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; @@ -48,12 +53,6 @@ import org.junit.Ignore; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; - -import static org.junit.Assert.assertTrue; - /** * Simple test suite for RBK. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java index 9fee23eeba998..ddac9f5af937f 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam.testkit; +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.beam.BeamExecutor; import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorEnvironment; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import java.time.Duration; - /** * Local {@link ExecutorProvider}. */ diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java index a8d27582d8937..2a33cfae35a47 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java @@ -25,9 +25,10 @@ import org.junit.runners.Suite; /** - * This is a copy of {@link org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite} to allow us track - * progress on incrementally implementing operator and their tests. TODO: When done, this class - * should go away and original should be used instead + * This is a copy of + * {@link org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite} to allow us track + * progress on incrementally implementing operator and their tests. + * TODO: When done, this class should go away and original should be used instead */ @RunWith(ExecutorProviderRunner.class) @Suite.SuiteClasses({ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java index 5a75734f820cc..01e8a9555f315 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/AccumulatorProvider.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import java.io.Serializable; - /** * Provides access to an accumulator backend service. It is intended to be implemented by third * party to support different type of services. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java index dde50ab2c7f30..538b908fceaca 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Timer.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.time.Duration; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Timer provides convenience API very similar to {@link Histogram} but extended by time unit diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java index 5afa39dfa1bc4..2d760bf8a23ba 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/VoidAccumulatorProvider.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.Duration; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Placeholder implementation of {@link AccumulatorProvider} that may be used in executors as a * default. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java index 2fcd89dfe1e5e..22d858cebb1ab 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/Dataset.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; +import java.io.Serializable; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import javax.annotation.Nullable; -import java.io.Serializable; -import java.util.Collection; - /** * A dataset abstraction. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java index 9e2a99d0464f6..49c147fcdf734 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/InputDataset.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import javax.annotation.Nullable; -import java.util.Collection; - /** {@code InputDataset} that is input of a {@code Flow}. */ @Audience(Audience.Type.EXECUTOR) class InputDataset implements Dataset { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java index 4b27e62034048..8e93eee230475 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/OutputDataset.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import javax.annotation.Nullable; -import java.util.Collection; - /** {@code OutputDataset} that is output of some operator. */ @Audience(Audience.Type.EXECUTOR) class OutputDataset implements Dataset { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java index 4c01a32b07723..b88c3bd74ded5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Count.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import static java.util.Collections.singleton; + import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import static java.util.Collections.singleton; - /** Count tumbling windowing. */ @Audience(Audience.Type.CLIENT) public final class Count implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java index 5eec4e2f40566..6581fab01b7e0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/GlobalWindowing.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import java.io.ObjectStreamException; +import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import java.io.ObjectStreamException; -import java.util.Collections; - /** Windowing with single window across the whole dataset. Suitable for batch processing. */ @Audience(Audience.Type.CLIENT) public final class GlobalWindowing implements Windowing { @@ -63,8 +62,9 @@ public int hashCode() { } /** Singleton window. */ - public static final class Window - extends org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window { + public static final class Window extends + org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window< + GlobalWindowing.Window> { static final Window INSTANCE = new Window(); diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java index e41f3d16909a0..c17acb31dcbd1 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/MergingWindowing.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import java.util.Collection; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import java.util.Collection; - /** * @param * @param diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java index c085db6b85489..168ef1d6517e6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Session.java @@ -17,15 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.AfterFirstCompositeTrigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.PeriodicTimeTrigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import static com.google.common.base.Preconditions.checkArgument; -import javax.annotation.Nullable; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -34,8 +27,14 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; - -import static com.google.common.base.Preconditions.checkArgument; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.AfterFirstCompositeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.PeriodicTimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** Session windowing. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java index b55c70b9ba39a..ce3758f33d4ba 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Time.java @@ -17,20 +17,19 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Collections.singleton; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.AfterFirstCompositeTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.PeriodicTimeTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import javax.annotation.Nullable; -import java.time.Duration; -import java.util.Arrays; -import java.util.Objects; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Collections.singleton; - /** Time based tumbling windowing. Windows can't overlap. */ @Audience(Audience.Type.CLIENT) public class Time implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java index bc09047e1e462..eeb8cc2e1a1b9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSliding.java @@ -17,18 +17,17 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import static com.google.common.base.Preconditions.checkArgument; + import com.google.common.collect.AbstractIterator; +import java.time.Duration; +import java.util.Iterator; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TimeTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import java.time.Duration; -import java.util.Iterator; -import java.util.Objects; - -import static com.google.common.base.Preconditions.checkArgument; - /** Time sliding windowing. */ @Audience(Audience.Type.CLIENT) public final class TimeSliding implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java index 4e731bfe14d65..ac8952c173cb3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Window.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A {@link Windowing} strategy associates each input element with a window thereby grouping input diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java index e810d579bf448..5ad6df3b60200 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/Windowing.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import java.io.Serializable; - /** A windowing policy of a dataset. All implementations must implement equals/hashCode. */ @Audience(Audience.Type.CLIENT) public interface Windowing> extends Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java index 7357b3d2274bb..cc002474c85db 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Flow.java @@ -17,18 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.flow; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; import java.io.IOException; import java.io.ObjectOutputStream; import java.io.OutputStream; @@ -42,6 +30,17 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ExtractEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** A dependency graph of operators. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java index ffc91f8d3e5dd..e641a009d46e7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunction.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Function of two arguments. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java index eff3305762b43..3f1b28bb1a7c4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import java.io.Serializable; - /** Functor of two arguments. */ @Audience(Audience.Type.CLIENT) @FunctionalInterface diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java index 86b8813eb5dd6..6d426ea5856b9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A consumer of given type. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java index 3f95a90a67f32..8d21713eac9ee 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** @param */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java index 988bedf3c1391..f1ed706dd9359 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Reduce function reducing iterable of elements into single element (of possibly different type). diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java index 721d777343057..c596bdb79e7a8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunctor.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Reduce function reducing iterable of elements into multiple elements (of possibly different diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java index b712f7b55efb4..e70009b2bbacb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Function taking three arguments. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java index 0d742c87d59d6..7fb831dd42904 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; -import java.io.Serializable; - /** @param */ @Audience(Audience.Type.INTERNAL) public interface TypeHintAware extends Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java index 5fe69656eeb54..a45259a97ff4b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunction.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Function of single argument. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java index 3b3953839ea21..ae39e66349d59 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctionEnv.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import java.io.Serializable; - /** * Function of single argument with access to Euphoria environment via context. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java index 3292b4fb94f2c..d239c9e2a8643 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryFunctor.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import java.io.Serializable; - /** * Functor of single argument. Functor can produce zero or more elements in return to a call, for * which it uses a collector. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java index 1862726abf2ec..8e2a7c9d996a0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.functional; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Function taking zero arguments. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java index 12e41c44af3d5..34b5b8f44b55c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedDataSource.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.IOException; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A {@code DataSource} with bounded data. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java index e93fab03e5607..eaf9d9b85c51f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Reader of bounded data. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java index f32e65c964819..07a8c7a14f8eb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Closeable; import java.util.Iterator; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** {@link Iterator} that is {@link Closeable}. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java index b8a01fd8650bb..01ae2789cb86a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSink.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; - import java.io.IOException; import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; /** * Sink for a dataset. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java index d493c240409f1..5de8bac9bdec3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSinks.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; +import java.io.IOException; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; -import java.io.IOException; - /** Various {@link DataSink} related utilities. */ public class DataSinks { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java index ae86930ca38bc..4c045929bfebe 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/DataSource.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Source of data for dataset. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java index 6f97645a6faca..a6bcd85525e89 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Closeable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** An {@code Iterable} that is externalized to external storage (i.e. spilled to disk). */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java index 6ac12de28223b..2d114cbd3560f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSink.java @@ -17,11 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.Consumer; - -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -30,6 +25,10 @@ import java.util.Map; import java.util.WeakHashMap; import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.Consumer; /** A data sink that stores data in list. */ @Audience({Audience.Type.CLIENT, Audience.Type.TESTS}) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java index aa9d763903387..a5eaa4d89fa33 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSource.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.io; import com.google.common.collect.Lists; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -32,6 +30,7 @@ import java.util.WeakHashMap; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A {@code DataSource} that is backed up by simple list. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java index 260b60269d063..e787b49c43950 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSink.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; /** * MultiDataSink allows to save to multiple {@link DataSink}. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java index 1a43c05eb21f0..45b23341283f0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/SpillTools.java @@ -19,14 +19,13 @@ import com.google.common.collect.Iterables; import com.google.common.io.Closeables; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; - import java.io.IOException; import java.io.Serializable; import java.util.Collection; import java.util.Comparator; import java.util.Iterator; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; /** Tools that can be used to externalize a dataset to local storage. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java index 0ab7a88fe2a56..311ea54e01295 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/StdoutSink.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.IOException; import java.io.PrintStream; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A sink to write to a specified print stream (typically {@link java.lang.System#out}) using the diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java index fa615129d6524..00f9a1c599943 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedDataSource.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; import java.util.List; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * {@code DataSource} for unbounded data. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java index 474467fd24b24..db9291e0d67cb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnboundedPartition.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.IOException; import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Single partition of a bounded dataset. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java index a918b5d17d7a5..2a9d566bd3cb6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/UnsplittableBoundedSource.java @@ -23,10 +23,9 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A {@code BoundedPartition} that cannot be split into smaller pieces. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java index 440a5ff5b8645..3a531dbe0a501 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/VoidSink.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.IOException; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** TODO: complete javadoc. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java index 3fa85b27ba1d7..8fe6b789d3346 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Writer.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Closeable; import java.io.IOException; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Writer for data to a particular partition. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java index 6225db81b7879..a990aadfe020e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/Split.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.lib; +import java.io.Serializable; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; -import java.io.Serializable; -import java.util.Objects; - /** * Composite operator using two {@link Filter} operators to split a {@link Dataset} into two subsets * using provided {@link UnaryPredicate}. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java index c1deb2d9101f1..7f206343a6572 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import java.util.Objects; -import java.util.Set; - /** * A convenient alias for assignment of event time. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java index 56e395d6a42ed..3abdbe0544fce 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java @@ -18,6 +18,9 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -30,10 +33,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import javax.annotation.Nullable; -import java.util.Objects; -import java.util.Set; - /** * Operator counting elements with same key. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java index 96c51fd92d0bf..1474d19d52955 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java @@ -18,6 +18,10 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -31,11 +35,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; - /** * Operator outputting distinct (based on {@link Object#equals}) elements. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java index 2c5546fd589ee..f55d41049c1a6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ElementWiseOperator.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import java.util.Set; - /** * Operator working element-wise, with no context between elements. No windowing scheme is needed to * be defined on input. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java index a57ef4085c26e..64a1ead3bab88 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import java.util.Objects; -import java.util.Set; - /** * Operator performing a filter operation. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java index 28c84bfde9d7c..14a274c3335af 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java @@ -18,6 +18,10 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -28,11 +32,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; - /** * A transformation of a dataset from one type into another allowing user code to generate zero, * one, or many output elements for a given input element. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java index c2dd55345787e..5a872506d866e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.Objects; +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import java.util.Objects; -import java.util.Optional; - /** * Full outer join of two input datasets producing single new dataset. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java index d819f36108187..04e9672876443 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java @@ -19,6 +19,11 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import java.util.Arrays; +import java.util.Collection; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -40,12 +45,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import javax.annotation.Nullable; -import java.util.Arrays; -import java.util.Collection; -import java.util.Objects; -import java.util.Set; - /** * Inner join of two datasets by given key producing single new dataset. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java index 35d27fecb54c0..4d44a97ddbc53 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.Objects; +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import java.util.Objects; -import java.util.Optional; - /** * Left outer join of two input datasets producing single new dataset. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java index 8cf6e28edc348..2c14340bce6df 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java @@ -18,6 +18,9 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -28,10 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; - /** * Simple one-to-one transformation of input elements. It is a special case of {@link FlatMap} with * exactly one output element for every one input element. No context is provided inside the map diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java index bf74ed5eb31cc..1397420998366 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Operator.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; @@ -24,12 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import java.io.Serializable; -import java.util.Collection; -import java.util.Set; - -import static com.google.common.base.Preconditions.checkArgument; - /** An operator base class. All operators inherit his class. */ @Audience(Audience.Type.INTERNAL) public abstract class Operator implements Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java index 662ed44bf5569..64860ee5c563a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/OptionalMethodBuilder.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import java.util.Objects; - /** * Class to be extended by operator builders that want to make use of `applyIf` call. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java index 3f20ab4170768..64bd988a2f0d5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java @@ -18,6 +18,12 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Comparator; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Recommended; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -49,13 +55,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; -import javax.annotation.Nullable; -import java.util.Comparator; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; - /** * Operator performing state-less aggregation by given reduce function. The reduction is performed * on all extracted values on each key-window. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java index 3c1fab841b56c..420e62e1b9d12 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKey.java @@ -18,10 +18,14 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; @@ -34,10 +38,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.type.TypeHint; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import javax.annotation.Nullable; -import java.util.Objects; -import java.util.Set; - /** * A {@link ReduceStateByKey} operator is a stateful, complex, lower-level-api, but very powerful * processor and serves as the basis for easier-to-use operators. Client API users have generally @@ -297,8 +297,8 @@ public static class DatasetBuilder4 extends Operator { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseOperator.java index 2d049ae05a768..e9bb14ae9cceb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseOperator.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import javax.annotation.Nullable; - /** Operator with internal state. */ @Audience(Audience.Type.INTERNAL) public abstract class StateAwareWindowWiseOperator< diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java index 5adba6a710ea6..98e23f79ab6e3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/StateAwareWindowWiseSingleInputOperator.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; @@ -25,11 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import javax.annotation.Nullable; -import java.util.Collection; -import java.util.Collections; -import java.util.Set; - /** Operator operating on window level with state information. */ @Audience(Audience.Type.INTERNAL) public class StateAwareWindowWiseSingleInputOperator< diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java index fc5c1da1933f7..7269c3514912d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java @@ -18,6 +18,10 @@ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -31,11 +35,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Objects; -import java.util.Set; - /** * Operator for summing of long values extracted from elements. The sum is operated upon defined key * and window. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java index 4e99e5d8719d5..a2c51bab280b6 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java @@ -17,7 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static java.util.Objects.requireNonNull; + import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Derived; import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; @@ -37,12 +42,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Set; - -import static java.util.Objects.requireNonNull; - /** * Emits top element for defined keys and windows. The elements are compared by comparable objects * extracted by user defined function applied on input elements. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java index 5d5be718f05e8..7938884a8d7f7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java @@ -17,22 +17,21 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; -import com.google.common.collect.Sets; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; +import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Sets; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; - -import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.Basic; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.operator.StateComplexity; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; /** * The union of at least two datasets of the same type. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java index d39217f5942d8..40a7a6be3560c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; -import javax.annotation.Nullable; - /** Operator working on some context. */ @Audience(Audience.Type.INTERNAL) public abstract class WindowWiseOperator> diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java index bdbe0b1e07067..fdd0ccb42b79e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Specify hints about runtime data characteristics. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateContext.java index 318bb31ed8b3a..0a0cf4294a07f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateContext.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; -import java.io.Serializable; - /** A context that is passed to stateful operators when creating {@code State}. */ @Audience(Audience.Type.CLIENT) public interface StateContext extends Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java index a360159223e67..247608a9679cb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; +import java.io.Serializable; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.annotation.stability.Experimental; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import javax.annotation.Nullable; -import java.io.Serializable; - /** Factory for states. */ @Audience(Audience.Type.CLIENT) @FunctionalInterface diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java index 5fb9df1046247..a757e416e90d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; /** * A function to merge specific types of states into a given target state. The need for merging - * states into one arise typically from the utilization of {@link - * org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing}, e.g. session windows, where - * individual session windows need occasionally be merged and, thus, their states. + * states into one arise typically from the utilization of {@link MergingWindowing}, e.g. session + * windows, where individual session windows need occasionally be merged and, thus, their states. * * @param the type of input elements for the states * @param the type of output elements of the states diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java index 5fb51455e360a..248db127dc2d3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Descriptor of storage. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java index 2ec8e68a9b7fb..0b33632246d39 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageProvider.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A provider of storage instances. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java index 398b80a838402..1c88c6de95dbb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/AfterFirstCompositeTrigger.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import java.util.List; - /** * Composite {@link Trigger} that fires once after at least one of sub-triggers have fired. In other * words sub-triggers are composed using logical OR. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java index bb6d9e0533261..3a14eaf66a6fd 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/triggers/Trigger.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.triggers; +import java.io.Serializable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageDescriptor; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import java.io.Serializable; - /** * Trigger determines when a window result should be flushed. * diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java index d4f45d272b78a..751f3f4a428ac 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/AbstractTypeAware.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.type; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.TypeHintAware; - import java.io.Serializable; import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.TypeHintAware; abstract class AbstractTypeAware implements Serializable, TypeHintAware { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java index f2f0145b588c9..fabacb3c92512 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeAwareReduceFunctor.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.type; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; -import java.util.stream.Stream; - /** TODO: complete javadoc. */ public class TypeAwareReduceFunctor extends AbstractTypeAware, OutT> implements ReduceFunctor { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java index ed05d3200a614..06800eee14a54 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeHint.java @@ -19,10 +19,9 @@ import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; - import java.io.Serializable; import java.lang.reflect.Type; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** TODO: complete javadoc. */ public abstract class TypeHint implements Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java index 441cab1eefb8e..e6d040644c8cb 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Either.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import static java.util.Objects.requireNonNull; import javax.annotation.Nullable; - -import static java.util.Objects.requireNonNull; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Either LeftT or RightT element. */ @Audience(Audience.Type.INTERNAL) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java index 2c8c9f406528f..68ff05153178a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Fold.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; +import java.io.Serializable; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; -import java.io.Serializable; -import java.util.stream.Stream; - /** Apply a folding function. */ public class Fold implements Serializable { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java index 8cb5eeeb861e2..44cfd03e35753 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Max.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; +import java.util.Comparator; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import java.util.Comparator; - /** Calculate maximum. */ @Audience(Audience.Type.CLIENT) public class Max { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java index c0b04a1cdaccc..e066f261a5434 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Pair.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.util.Comparator; import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A pair, i.e. a tuple of two elements. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java index bf1f2a0fe9d16..46cf1d32e7009 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Sums.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.CombinableReduceFunction; -import java.util.stream.Collectors; - /** Provides commonly used function objects around computing sums. */ @Audience(Audience.Type.CLIENT) public class Sums { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java index 385c1d40bd60e..00191518f7074 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/Triple.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Triple of any types. */ @Audience(Audience.Type.CLIENT) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java index 36f17d0b22b79..020d3918f1edf 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/AbstractExecutor.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor; -import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** TODO: complete javadoc. */ public abstract class AbstractExecutor implements Executor { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java index 6ab6ac5fd5232..291a6d08f0929 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/Executor.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.euphoria.core.executor; import com.google.common.collect.Sets; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; -import java.util.Set; -import java.util.concurrent.CompletableFuture; - /** The client side, public, technology independent interface to an executor. */ @Audience(Audience.Type.CLIENT) public interface Executor { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java index c3e4c3817e0e7..6e290f14b2509 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolder.java @@ -18,15 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.core.executor; import com.google.common.collect.Iterables; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -37,6 +28,14 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; /** Unfold {@code Flow} to contain only selected operators. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java index c489bb6b54144..e0084aee1aaf9 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowValidator.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor; +import static com.google.common.base.Preconditions.checkState; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; @@ -29,13 +35,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static com.google.common.base.Preconditions.checkState; - /** Validate invariants. Throw exceptions if any invariant is violated. */ class FlowValidator { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java index 7287f6094d702..57d55466c4189 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/VectorClock.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Vector clock implementation for local executor. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java index 80782292eb661..4e01486ec18f7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAG.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -35,6 +32,8 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** * A directed acyclic graph of nodes of type T. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java index 3d7dfe5a50b84..40d09137fb048 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/Node.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** A single Node in DAG. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java index 815d115ec3d06..9bfaf420989a5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/GroupReducer.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; +import static com.google.common.base.Preconditions.checkState; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; @@ -43,14 +50,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Objects; - -import static com.google.common.base.Preconditions.checkState; - /** * An implementation of a RSBK group reducer of an ordered stream of already grouped (by a specific * key) and windowed elements where no late-comers are tolerated. Use this class only in batch mode! @@ -265,7 +264,8 @@ public interface WindowedElementFactory { WindowedElement create(W window, long timestamp, T element); } - class ElementCollector implements Context, org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector { + class ElementCollector + implements Context, org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector { final Collector>> out; final WidT window; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java index c8c7822d578d5..77be84fc263de 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TimerSupport.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; - -import javax.annotation.Nullable; import java.util.HashSet; import java.util.PriorityQueue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * Keeps track of the current watermark within a reduce operation and provides services around diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java index 593787c15ac20..2b6ec3e215cc0 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/greduce/TriggerStorage.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.greduce; +import java.util.HashMap; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; @@ -26,9 +28,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import java.util.HashMap; -import java.util.Objects; - /** * Maintains triggers storages in memory for referencing through value descriptors in the scope of a * window. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java index 7d0b0f828213f..b5ee255088725 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorage.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; - import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; @@ -31,6 +27,9 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; /** * A list storage implementation keeping a given maximum number of received elements, eventually diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java index ce31cd4e8d4e1..92f216df89e44 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillTools.java @@ -18,14 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; import com.google.common.annotations.VisibleForTesting; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; -import org.apache.beam.sdk.extensions.euphoria.core.executor.Constants; -import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.util.ArrayList; import java.util.Collection; @@ -34,6 +26,13 @@ import java.util.UUID; import java.util.stream.Stream; import java.util.stream.StreamSupport; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.executor.Constants; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** An implementation of {@code SpillTools} to be used by executors. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java index dc6e01c57dce2..4fcbe8c45294c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SerializerFactory.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Closeable; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** TODO: complete javadoc. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java index 3fa923867c8f6..c10e925eb6b6e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/SpillFileFactory.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.File; import java.io.Serializable; import java.net.URI; import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.Path; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** Factory for creating files for spilling. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java index f55e88935a077..3cecc92ce7a74 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/InMemExternalIterable.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.util; +import java.util.Iterator; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import java.util.Iterator; - /** * An {@code ExternalIterable}, that is stored in memory. Use this class with care, because it might * cause OOME or other performance issues. diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java index 4d791ab50fff5..1b6d492071f4f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/MultiValueContext.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.util; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; @@ -25,11 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - /** TODO: complete javadoc. */ @Audience(Audience.Type.EXECUTOR) public class MultiValueContext implements Context, Collector { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java index 77e22a4f62a79..2cbfcb915bf54 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/OperatorTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.util; +import java.util.ArrayList; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; @@ -25,8 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; -import java.util.ArrayList; - /** Util class when specific executors use the same methods for operator translation. */ @Audience(Audience.Type.EXECUTOR) public class OperatorTranslator { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java index 3e75d02875e3f..679cfc2cfaa4b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/executor/util/SingleValueContext.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.util; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; @@ -25,8 +26,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; -import javax.annotation.Nullable; - /** * A {@code Context} that holds only single value. There is no window associated with the value, so * the {@code getWindow()} will always throw {@code UnsupportedOperationException}. This context diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java index 580ab9d8b85f6..0ce4674588a42 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.time; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.time.Duration; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** TODO: complete javadoc. */ @Audience(Audience.Type.EXECUTOR) diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java index 7a65959689bc2..6e79580612999 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvider.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.time; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.io.Serializable; import java.time.Duration; import java.util.Date; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A provider of the "current" timestamp aiming to decouple the "current" time consumers and their diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java index 71a3eb43014d7..93b66a26ec33f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviders.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.time; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.time.Duration; import java.util.Calendar; import java.util.Date; import java.util.Objects; import java.util.TimeZone; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * A set of pre-defined time provider implementations all based on utilities provided by the diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java index 0bf908572f3f4..12ab1a2cf6e05 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimerScheduler.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.time; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.time.Duration; import java.util.Timer; import java.util.TimerTask; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A scheduler implementation with an underlying timer periodically executing the specified diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java index 093ae93e5db61..b0903474d743e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.util; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; - import java.lang.reflect.Constructor; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; /** * Util class that helps instantiations of objects throwing {@link RuntimeException}. For core diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java index 32559d520628c..f43d60bbd3216 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/Settings.java @@ -17,17 +17,16 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.util; -import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import static java.util.Objects.requireNonNull; -import javax.annotation.Nullable; import java.io.Serializable; import java.net.URI; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; - -import static java.util.Objects.requireNonNull; +import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** * General utility class to store key/value pairs as strings providing converter methods for diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java index e79d1b3ce54b2..46d0b1175ab90 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/SessionTest.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import com.google.common.collect.Iterables; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import com.google.common.collect.Iterables; import java.time.Duration; import java.util.Arrays; import java.util.Collection; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; /** TODO: complete javadoc. */ public class SessionTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java index ce31d2da05656..0ee3c53b4efa7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeSlidingTest.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; +import static org.junit.Assert.assertEquals; + import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import org.junit.Test; - import java.time.Duration; - -import static org.junit.Assert.assertEquals; +import org.junit.Test; /** TODO: complete javadoc. */ public class TimeSlidingTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java index 6d591eedae822..6fb7809b0c142 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/TimeTest.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import com.google.common.collect.Iterables; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import com.google.common.collect.Iterables; import java.time.Duration; - -import static org.junit.Assert.assertEquals; +import org.junit.Test; /** TODO: complete javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java index 605fa0a2ee29b..61c1e4e6a24f7 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/dataset/windowing/WindowingTest.java @@ -17,18 +17,19 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing; -import com.google.common.collect.Iterables; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import com.google.common.collect.Iterables; import java.time.Duration; import java.util.Arrays; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; - -import static org.junit.Assert.*; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.junit.Test; /** TODO: complete javadoc. */ public class WindowingTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java index f475cff102cf8..200df92375356 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/TestFlow.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.flow; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.io.MockStreamDataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; @@ -25,14 +32,6 @@ import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.Collection; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.assertEquals; - /** Test some basic features of flow. */ public class TestFlow { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java index 8116c17ded672..e0158577b9847 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ListDataSinkTest.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; +import static org.junit.Assert.assertEquals; + import com.google.common.collect.Iterables; import org.apache.beam.sdk.extensions.euphoria.core.testing.DatasetAssert; import org.junit.Test; -import static org.junit.Assert.assertEquals; - /** TODO: complete javadoc. */ public class ListDataSinkTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java index c363019ba2e52..839eaeafd2128 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/MultiDataSinkTest.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.io; -import org.junit.Test; -import org.mockito.Mockito; - -import java.io.IOException; - import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.io.IOException; +import org.junit.Test; +import org.mockito.Mockito; + /** Tests behavior of {@link MultiDataSink}. */ public class MultiDataSinkTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java index 8c18276dabafa..1c2c7b5a8b416 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/lib/SplitTest.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.lib; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryPredicate; @@ -25,10 +32,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Util; import org.junit.Test; -import java.util.Optional; - -import static org.junit.Assert.*; - /** TODO: complete javadoc. */ public class SplitTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java index be6d5daa80a14..ca00a5f434aad 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKeyTest.java @@ -17,16 +17,18 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; -import java.time.Duration; - -import static org.junit.Assert.*; - /** Test operator CountByKey. */ public class CountByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java index 38187617d02ac..bd84d4b945b7d 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/DistinctTest.java @@ -17,15 +17,16 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.junit.Test; -import java.time.Duration; - -import static org.junit.Assert.*; - /** Test operator Distinct. */ public class DistinctTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java index 0f81ba834cf49..14a02fe199542 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FilterTest.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - /** Test operator Filter. */ public class FilterTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java index 6b4ccd67d91f3..7557ad53b8404 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java @@ -17,15 +17,16 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.math.BigDecimal; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.junit.Test; -import java.math.BigDecimal; - -import static org.junit.Assert.*; - /** Test operator FlatMap. */ public class FlatMapTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java index 2427e8b00516f..93ba6d03392b4 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/HintTest.java @@ -17,7 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.Set; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; @@ -32,12 +37,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; import org.junit.Test; -import java.time.Duration; -import java.util.Set; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** Test usage of hints in different operators. */ public class HintTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java index 09ffd96df793b..2fabc7903f2d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; @@ -25,11 +32,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; -import java.time.Duration; -import java.util.Optional; - -import static org.junit.Assert.*; - /** Test operator Join. */ public class JoinTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java index 25c27e066fdac..301e7fa173152 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElementsTest.java @@ -17,13 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.SizeHint; import org.junit.Test; -import static org.junit.Assert.*; - /** Test operator MapElement. */ public class MapElementsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java index 80429c2956eb0..2e05927aaa9b5 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java @@ -17,17 +17,20 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.stream.StreamSupport; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; -import java.time.Duration; -import java.util.stream.StreamSupport; - -import static org.junit.Assert.*; - /** Test operator ReduceByKey. */ public class ReduceByKeyTest { @Test diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java index 83386c3aa4702..09e0f47866937 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceStateByKeyTest.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; @@ -28,10 +34,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; -import java.time.Duration; - -import static org.junit.Assert.*; - /** Test operator ReduceStateByKey. */ public class ReduceStateByKeyTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java index 57a77ba541bdb..c18fa02588d2b 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindowTest.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; @@ -25,11 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.executor.util.SingleValueContext; import org.junit.Test; -import java.time.Duration; -import java.util.stream.Stream; - -import static org.junit.Assert.*; - /** Test behavior of operator {@code ReduceWindow}. */ public class ReduceWindowTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java index a47d65b2e967a..73069fc20d5de 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java @@ -17,16 +17,18 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; -import java.time.Duration; - -import static org.junit.Assert.*; - /** Test behavior of operator {@code SumByKey}. */ public class SumByKeyTest { @Test diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java index 961117e30320c..e784df75a595a 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKeyTest.java @@ -17,17 +17,19 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Iterables; +import java.time.Duration; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; import org.junit.Test; -import java.time.Duration; - -import static org.junit.Assert.*; - /** Test behavior of operator {@code TopPerKey}. */ public class TopPerKeyTest { @Test diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java index aae48d4a7bd76..7dc9cdd11b130 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/UnionTest.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import static org.junit.Assert.assertEquals; + import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.junit.Test; -import static org.junit.Assert.assertEquals; - /** Test behavior of operator {@code Union}. */ public class UnionTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java index 70d78505a3298..5466025cce3da 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Util.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.operator; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint.OutputHint; -import java.util.ArrayList; -import java.util.List; - /** Utility class for easier creating input datasets for operator testing. */ public class Util { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java index 0c6fba1ab7d05..8226044232c36 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.client.util; -import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; import java.util.stream.Stream; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; +import org.junit.Test; /** Test behavior of IOUtils. */ public class IOUtilsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java index 54c02c3719aa9..5fc251a97a8dc 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/FlowUnfolderTest.java @@ -17,8 +17,20 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Datasets; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -42,17 +54,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.junit.Assert.*; - /** {@code FlowUnfolder} test suite. */ public class FlowUnfolderTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java index 3808c9e5e8239..1dce1d25c1cd8 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/graph/DAGTest.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.graph; -import com.google.common.collect.Iterables; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import com.google.common.collect.Iterables; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.Test; /** DAG test suite. */ public class DAGTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java index 261e15c03bf7f..f644c06e2ce93 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/FsSpillingListStorageTest.java @@ -17,20 +17,21 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import com.google.common.collect.Lists; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; import java.util.stream.StreamSupport; - -import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; /** TODO: complete javadoc. */ public class FsSpillingListStorageTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java index 8cb85e835e39f..0eb8cd7cc630f 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/GenericSpillToolsTest.java @@ -17,11 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.util.Collection; import java.util.Comparator; @@ -29,9 +26,11 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.StreamSupport; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; /** Test suite for {@code GenericSpillTools}. */ public class GenericSpillToolsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java index 0ad31c4b2da77..1c6e26d99460e 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/executor/io/TmpFolderSpillFileFactory.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.executor.io; -import org.junit.rules.TemporaryFolder; - import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import org.junit.rules.TemporaryFolder; /** TODO: complete javadoc. */ class TmpFolderSpillFileFactory implements SpillFileFactory { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java index e65d337b847ff..78786621f58f3 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testing/DatasetAssert.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.testing; -import org.junit.Assert; - import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.junit.Assert; /** This is duplicated from {@code euphoria-testing} due to maven lifecycle cyclic dependency. */ public class DatasetAssert { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java index c0dffebd736b4..b5dfdb000316c 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProvidersTest.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.time; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.time.Duration; import java.util.Calendar; import java.util.Date; import java.util.TimeZone; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.junit.Test; /** TODO: complete javadoc. */ public class TimeProvidersTest { diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java index 7b226fe002c36..7232d7e964091 100644 --- a/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-core/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/util/SettingsTest.java @@ -17,10 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.core.util; -import org.junit.Test; - import static org.junit.Assert.assertEquals; +import org.junit.Test; + /** TODO: complete javadoc. */ public class SettingsTest { diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java index 65f4b3819fc7d..83bb9bb73a817 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.fluent; +import static java.util.Objects.requireNonNull; + import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; @@ -27,8 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Union; import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; -import static java.util.Objects.requireNonNull; - /** TODO: complete javadoc. */ public class Dataset { @@ -43,7 +43,9 @@ public org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset un } public Dataset apply( - UnaryFunction, Output> output) { + UnaryFunction< + org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset, + Output> output) { return new Dataset<>(requireNonNull(output.apply(this.wrap)).output()); } diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java index 73df1e8d2a543..3bad553228104 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.fluent; +import static java.util.Objects.requireNonNull; + import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import static java.util.Objects.requireNonNull; - /** TODO: complete javadoc. */ public class Flow { private final org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow wrap; @@ -35,7 +35,8 @@ public static Flow create(String name) { } public static Flow create(String name, Settings settings) { - return new Flow(org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow.create(name, settings)); + return new Flow( + org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow.create(name, settings)); } public Dataset read(DataSource src) { diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java index dd0c7cc006556..6110a5e380b91 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java @@ -30,7 +30,8 @@ public static Flow flow(String name, Settings settings) { return Flow.create(name, settings); } - public static Dataset lift(org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset xs) { + public static Dataset lift( + org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset xs) { return new Dataset<>(xs); } } diff --git a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java index 2d861b0889889..ba2402f6a95fc 100644 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/test/java/org/apache/beam/sdk/extensions/euphoria/fluent/FluentTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.fluent; +import static java.util.Arrays.asList; + +import java.time.Duration; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; @@ -26,12 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.executor.local.LocalExecutor; import org.junit.Test; -import java.time.Duration; -import java.util.Set; -import java.util.stream.Collectors; - -import static java.util.Arrays.asList; - /** Test behavior of Fluent API. */ public class FluentTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java index 10c73129f10f8..18fc39ab00295 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AbstractTriggerScheduler.java @@ -21,9 +21,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.ObjectInputStream; import java.util.ArrayList; @@ -34,6 +31,8 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Base class for various time triggering strategies. */ public abstract class AbstractTriggerScheduler implements TriggerScheduler { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java index 7881d6a707ade..f2665c747fd5d 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/AttachedWindowing.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import java.util.Collections; - class AttachedWindowing> implements Windowing { static final AttachedWindowing INSTANCE = new AttachedWindowing(); diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java index adc7fe3ffd4d1..8617ebad5f058 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecUnit.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; -import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; -import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.DAG; +import org.apache.beam.sdk.extensions.euphoria.core.executor.graph.Node; /** * TODO: this description is WRONG! An {@code ExecUnit} is a series of transformation with no diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java index ecb5d1951233d..729cd4437448f 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/KeyedWindow.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; - import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; /** * TODO: add javadoc. diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java index a3e4647d12f3e..2523ff7159cb0 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutor.java @@ -18,6 +18,32 @@ package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.VoidAccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; @@ -51,33 +77,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import java.util.stream.Stream; - /** Local executor for testing and not fault tolerant local applications. */ public class LocalExecutor implements Executor { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java index b0abca9b36148..c26964aef0d6b 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalSpillTools.java @@ -18,16 +18,15 @@ package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; -import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; -import org.apache.beam.sdk.extensions.euphoria.core.executor.util.InMemExternalIterable; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.SpillTools; +import org.apache.beam.sdk.extensions.euphoria.core.executor.util.InMemExternalIterable; /** {@code SpillTools} that actually don't spill and use memory instead. */ class LocalSpillTools implements SpillTools { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java index 1ae8f3e27463a..6deeaf8ecba86 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalStorageProvider.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorage; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ListStorageDescriptor; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StorageProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import java.util.ArrayList; -import java.util.List; - /** Provider of state storage for local executor. */ public class LocalStorageProvider implements StorageProvider { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java index b976f8ce76293..88d092c5a4ed5 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ReduceStateByKeyReducer.java @@ -17,6 +17,21 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.function.Supplier; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; @@ -44,22 +59,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.function.Supplier; - -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toSet; - class ReduceStateByKeyReducer implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ReduceStateByKeyReducer.class); diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java index 666071e7f2f13..b961b08d52716 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkEmitStrategy.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.util.concurrent.ThreadFactoryBuilder; - import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java index c8ab1be2259b1..bc1798cef0436 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WatermarkTriggerScheduler.java @@ -18,9 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.executor.local; import com.google.common.collect.Iterables; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; - import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -30,6 +27,8 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; /** Trigger scheduler based on watermarks. Uses event-time instead of real wall-clock time. */ public class WatermarkTriggerScheduler implements TriggerScheduler { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java index 5c1937a289555..d77c70b84f93d 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowedElementCollector.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import java.util.Objects; +import java.util.function.Supplier; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; @@ -26,9 +28,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.io.Context; import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; -import java.util.Objects; -import java.util.function.Supplier; - class WindowedElementCollector implements Context, Collector { private final org.apache.beam.sdk.extensions.euphoria.executor.local.Collector wrap; private final Supplier stampSupplier; diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java index 537e3f5dc5a7a..2d40a13dd3926 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/BasicOperatorTest.java @@ -17,7 +17,19 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static java.util.Arrays.asList; +import static java.util.stream.Collectors.toList; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -40,19 +52,6 @@ import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.junit.Test; -import java.time.Duration; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Arrays.asList; -import static java.util.stream.Collectors.toList; -import static org.junit.Assert.assertTrue; - /** Test basic operator functionality and ability to compile. */ public class BasicOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java index fee292b68e33e..7b4db6b9c137d 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/JoinOperatorTest.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static java.util.Arrays.asList; + +import java.time.Duration; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -36,11 +40,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.util.List; - -import static java.util.Arrays.asList; - /** * Test of Join operator. */ diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java index 5ebcf453c9c15..cf1c79a7b36aa 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/LocalExecutorTest.java @@ -17,8 +17,27 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -55,22 +74,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static org.junit.Assert.*; - /** * {@code LocalExecutor} test suite. The {@code LocalExecutor} stands on the basic operators, so we * just need to test it correctly implements all of them. Next we need to test that it can process diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java index 793effa077839..5fd22644bc3d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/VectorClockTest.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static org.junit.Assert.assertEquals; + import org.apache.beam.sdk.extensions.euphoria.core.executor.VectorClock; import org.junit.Test; -import static org.junit.Assert.assertEquals; - /** Test vector clocks. */ public class VectorClockTest { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java index edfccfcdcefca..f4a6f5da0ecf7 100644 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/WindowingTest.java @@ -17,7 +17,17 @@ */ package org.apache.beam.sdk.extensions.euphoria.executor.local; +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.Comparator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -39,17 +49,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; -import java.util.Comparator; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; - -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Collection of windowing teats. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java index 50053af0163b1..4ca7c4aaf177f 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/BroadcastHashJoinTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin; @@ -28,10 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - /** * Collection of broadcast hash join tests. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java index 4246448175657..933f02b0f7c35 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/CountByKeyTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; @@ -27,10 +30,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - /** Test operator {@code CountByKey}. */ @Processing(Type.ALL) public class CountByKeyTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java index 849cf1d43c29f..8f64c43eb4e84 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/DistinctTest.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; @@ -27,11 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; import org.junit.Test; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** Test for the {@link Distinct} operator. */ @Processing(Type.ALL) public class DistinctTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java index d05d5bbb9c3d6..974c1df67dc52 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FilterTest.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Filter; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** Test operator {@code Filter}. */ @Processing(Processing.Type.ALL) public class FilterTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java index a312a07fe18dd..24f224085d09c 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/FlatMapTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; @@ -26,12 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - /** Test operator {@code FlatMap}. */ @Processing(Processing.Type.ALL) public class FlatMapTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java index 960c67464b2bf..e05203fe0b668 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java @@ -17,6 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -42,15 +50,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import static org.junit.Assert.assertEquals; - /** Test operator {@code Join}. */ @Processing(Processing.Type.ALL) public class JoinTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java index f6c111ceed61e..2e8bc5157b2f2 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinWindowEnforcementTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; @@ -33,12 +38,6 @@ import org.junit.Test; import org.junit.runners.Parameterized; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * A collection of test methods. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java index ea880845fb891..9fead76103157 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/MapElementsTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunctionEnv; @@ -26,12 +31,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - /** Tests for operator {@code MapElements}. */ @Processing(Processing.Type.ALL) public class MapElementsTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index 75943d2edde34..e23ceb6d25780 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -17,8 +17,24 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; @@ -54,21 +70,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.*; - /** Test operator {@code ReduceByKey}. */ @Processing(Processing.Type.ALL) public class ReduceByKeyTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java index 42a49f4f7d66e..fa01abca544b8 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceStateByKeyTest.java @@ -17,7 +17,18 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import com.google.common.collect.Lists; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; @@ -49,18 +60,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** Test operator {@code ReduceStateByKey}. */ @Processing(Processing.Type.ALL) public class ReduceStateByKeyTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java index e901a25e759b4..7b9a3d66fc76f 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceWindowTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.AssignEventTime; @@ -26,10 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - /** Test operator {@code ReduceByKey}. */ @Processing(Processing.Type.ALL) public class ReduceWindowTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java index 581e2d20026b4..f0fe8693f8a40 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SinkTest.java @@ -17,6 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; @@ -28,16 +37,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - /** Test that a sub-flow applied on sink is correctly preserved. */ public class SinkTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java index b9975696629d3..a18aa1af4999a 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/SumByKeyTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.SumByKey; @@ -25,10 +28,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - /** Test operator {@code SumByKey}. */ @Processing(Processing.Type.ALL) public class SumByKeyTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java index 6a2b111377a48..97756244d7d7a 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/TopPerKeyTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static java.util.Arrays.asList; + +import java.io.Serializable; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.TopPerKey; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; @@ -24,12 +29,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.io.Serializable; -import java.util.List; -import java.util.Objects; - -import static java.util.Arrays.asList; - /** * TODO: add javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java index 156148dffcf15..18c06d3f830cf 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/UnionTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; @@ -25,9 +27,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** Test for operator {@code Union}. */ @Processing(Processing.Type.ALL) public class UnionTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java index 1079ca388f4ca..2292d9f85b860 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/Util.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; - import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; import java.util.List; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; class Util { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java index 2212bf5e049b5..b74257b1ef0d3 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WatermarkTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; @@ -30,10 +33,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; - /** * TODO: add javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java index 2f13ce9b205eb..c8fd5b5e5fa5d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/WindowingTest.java @@ -17,6 +17,16 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; @@ -46,17 +56,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; import org.junit.Test; -import java.time.Duration; -import java.time.Instant; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - /** Tests capabilities of {@link Windowing}. */ @Processing(Processing.Type.ALL) public class WindowingTest extends AbstractOperatorTest { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java index 2014c756811ae..0f04d931c3af9 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongCounter.java @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; - import java.util.concurrent.atomic.AtomicLong; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; final class LongCounter implements Counter, Snapshotable { diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java index e914c9743f3f1..9eb809a7041c0 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/LongHistogram.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; - import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; final class LongHistogram implements Histogram, diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java index a507946453a16..13919e0561061 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/NanosecondTimer.java @@ -18,11 +18,10 @@ package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; import com.google.common.collect.Maps; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; - import java.time.Duration; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; final class NanosecondTimer implements Timer, diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java index 55170e7572817..e66f81496d08d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SingleJvmAccumulatorProvider.java @@ -17,19 +17,18 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Accumulator; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; -import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; -import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; - import java.io.ObjectStreamException; import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Accumulator; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Counter; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Histogram; +import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer; +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; /** * An accumulator provider gathering accumulators in-memory. diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java index 7c47ef9244855..a4dba44914a57 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/AbstractOperatorTest.java @@ -17,6 +17,15 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSource; @@ -29,16 +38,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutionException; - -import static com.google.common.base.Preconditions.checkNotNull; -import static org.junit.Assert.assertEquals; - /** * Base class for test description of a test case. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java index 3ca7c217595f0..5f9596d44e20f 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProviderRunner.java @@ -17,6 +17,16 @@ */ package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; +import static com.google.common.base.Preconditions.checkArgument; + +import java.lang.reflect.AnnotatedElement; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing.Type; import org.junit.Ignore; import org.junit.Test; @@ -33,17 +43,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.lang.reflect.AnnotatedElement; -import java.lang.reflect.Modifier; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Stream; - -import static com.google.common.base.Preconditions.checkArgument; - /** * TODO: add javadoc. */ diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java index 787104611d4c6..e68543439ebf3 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/Processing.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.extensions.euphoria.operator.test.junit; import com.google.common.collect.Lists; - import java.lang.annotation.Inherited; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java index 3697ed8a380f9..65df5a7608ead 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.extensions.euphoria.testing; +import java.util.List; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; -import java.util.List; - /** * Abstract test class for user's {@link Flow} testing. * diff --git a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java index 2e0b7abf2e65d..a57cc184ce062 100644 --- a/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java +++ b/sdks/java/extensions/euphoria/euphoria-testing/src/main/java/org/apache/beam/sdk/extensions/euphoria/testing/DatasetAssert.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.extensions.euphoria.testing; -import org.junit.Assert; - import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.junit.Assert; /** * Test wide utility methods for dataset assertion.