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/org/apache/beam/sdk/extensions/euphoria/beam/BeamExecutor.java similarity index 69% 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 97524a80767a2..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 75% 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 592981762ec01..4f180a2672840 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,30 +15,11 @@ * 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 static java.util.stream.Collectors.toList; 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; @@ -45,6 +28,25 @@ 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; 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 85% 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 397c3eae2a22f..ff5975dae7d6d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,18 +16,9 @@ * 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 java.time.Duration; import java.util.HashMap; @@ -34,6 +27,15 @@ 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; +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; 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 63% 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 3c84848e1db3d..ca902ab0af2b2 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +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.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 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; +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; /** 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 76% 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 2e4f6ef63b402..413a5737b279d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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; 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 69% 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 c93eadf327559..e03a65e424a89 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,25 +15,25 @@ * 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; 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 66% 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 cded0dd20c949..67d04159cded3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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; 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 62% 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 88fbfc126538f..f42ca2e3ffee5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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.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; +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; /** * Instantiate accumulator provider on the first usage. 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 64% 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 0c7a6dea7b7be..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 84% 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 ec459e052981f..4104044038f21 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +15,19 @@ * 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; +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; +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; 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 80% 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 3899f02f0af49..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 63% 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 feb59c7472922..7de542a532e47 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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.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; +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; /** * {@code Collector} for combinable functors. 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 68% 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 cbded9d84d2c8..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 64% 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 d2a08241932a9..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 66% 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 f0534e52e370e..bf65c340fb2bc 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,13 +16,13 @@ * 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.Datasets; -import cz.seznam.euphoria.core.client.operator.Operator; 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; /** 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 85% 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 527650a040348..2a79c04822f3d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,9 +15,8 @@ * 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; @@ -24,6 +25,7 @@ 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; 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 79% 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 0b947103d28a0..4abca49b3ce5d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +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 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.extensions.euphoria.core.client.io.BoundedDataSource; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -57,7 +59,8 @@ 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 87% 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 daa7f9311196d..e10548aa163b0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,9 +15,8 @@ * 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.UnboundedDataSource; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -24,6 +25,7 @@ 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; @@ -94,7 +96,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 77% 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 28ed2cc50db0e..5c94206009ea7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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; 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 85% 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 d91b82b418240..e2679109c16b4 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,11 @@ * 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 java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -30,6 +31,7 @@ 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; /** 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 73% 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 716574c5b080c..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 78% 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 8c3589cc71420..7d409afdbede7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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; 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 64% 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 e98bdad9bd8fd..4f2036371263a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.WindowedElement; 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; /** * A {@code WindowedElement} created from Beam's element. 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/testkit/BeamOperatorsTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java deleted file mode 100644 index c800aa753fd9c..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/testkit/BeamOperatorsTest.java +++ /dev/null @@ -1,23 +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.beam.testkit; - -/** - * TODO: write javadoc. - */ -public class BeamOperatorsTest extends BeamOperatorsSuite implements BeamExecutorProvider { - -} 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 82% 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 cf09fde51b61a..3d4220c988c2f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,27 +16,27 @@ * 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; 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 62% 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 e8d89646dc134..26164e3607440 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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.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; 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 74% 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 6c9758c07cfaa..2a913ab9746a5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,39 +15,39 @@ * 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 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; +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; 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 60% 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 9bb875a31f03a..ddac9f5af937f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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; 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 52% 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 11a95aa0f3592..2a33cfae35a47 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +15,20 @@ * 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 - * 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-beam/src/test/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..33e8b246d35d6 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsTest.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.beam.testkit; + +/** + * TODO: write javadoc. + */ +public class BeamOperatorsTest extends BeamOperatorsSuite implements BeamExecutorProvider { + +} 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/client/accumulators/Accumulator.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java deleted file mode 100644 index 49cd104260c2c..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Accumulator.java +++ /dev/null @@ -1,27 +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.core.client.accumulators; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** - * Accumulators collect values from user functions. Accumulators allow user to calculate statistics - * during the flow execution. - * - *

Accumulators are inspired by the Hadoop/MapReduce counters. - */ -@Audience(Audience.Type.INTERNAL) -public interface Accumulator {} 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 deleted file mode 100644 index 28ed9716e18a1..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/accumulators/Counter.java +++ /dev/null @@ -1,33 +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.core.client.accumulators; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Counter is a type of accumulator making a sum from integral numbers. */ -@Audience(Audience.Type.CLIENT) -public interface Counter extends Accumulator { - - /** - * Increment counter by given value. - * - * @param value Value to be added to the counter. - */ - void increment(long value); - - /** Increment counter by one. */ - void increment(); -} 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/Util.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java deleted file mode 100644 index f2e4a6c9baf27..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/flow/Util.java +++ /dev/null @@ -1,41 +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.core.client.flow; - -import javax.annotation.Nullable; - -class Util { - - private Util() {} - - /** - * Trim string (instead of empty string return null). - * - * @param s input string - * @return non-empty trimmed string or null - */ - @Nullable - static String trimToNull(@Nullable String s) { - if (s == null) { - return null; - } - s = s.trim(); - if (s.isEmpty()) { - return null; - } - return s; - } -} 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/BinaryFunctor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java deleted file mode 100644 index 8b05f3004730c..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/BinaryFunctor.java +++ /dev/null @@ -1,28 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Collector; -import java.io.Serializable; - -/** Functor of two arguments. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface BinaryFunctor extends Serializable { - - void apply(LeftT left, RightT right, Collector context); -} 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 deleted file mode 100644 index f1fa000a95218..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableBinaryFunction.java +++ /dev/null @@ -1,23 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Function of two arguments of the same type. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface CombinableBinaryFunction extends SameArgsBinaryFunction {} 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 deleted file mode 100644 index 8dd41d2108847..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/CombinableReduceFunction.java +++ /dev/null @@ -1,26 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** - * Function reducing stream of elements into single one of the same type. The applied function has - * to be commutative associative. - */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface CombinableReduceFunction extends ReduceFunction {} 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 deleted file mode 100644 index deebe8d8006d3..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/Consumer.java +++ /dev/null @@ -1,32 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** A consumer of given type. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface Consumer extends Serializable { - - /** - * Consume given element. - * - * @param element the element to be consumed. - */ - void accept(T element); -} 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 deleted file mode 100644 index 53bf8415c9404..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ExtractEventTime.java +++ /dev/null @@ -1,33 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** @param */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface ExtractEventTime extends Serializable { - - /** - * Extracts event time (in millis since epoch) of the given element. - * - * @param elem input element - * @return the extracted event time - */ - long extractTimestamp(InputT elem); -} 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 deleted file mode 100644 index b16e090df2fac..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/ReduceFunction.java +++ /dev/null @@ -1,26 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.util.stream.Stream; - -/** - * Reduce function reducing iterable of elements into single element (of possibly different type). - */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface ReduceFunction extends UnaryFunction, OutputT> {} 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 deleted file mode 100644 index 968ab5fb0b210..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/SameArgsBinaryFunction.java +++ /dev/null @@ -1,24 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Function of two arguments of the same type. */ -@Audience(Audience.Type.INTERNAL) -@FunctionalInterface -public interface SameArgsBinaryFunction - extends BinaryFunction {} 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 deleted file mode 100644 index 01014fd8274af..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TernaryFunction.java +++ /dev/null @@ -1,27 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** Function taking three arguments. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface TernaryFunction extends Serializable { - - OutputT apply(FirstT first, SecondT second, ThirdT third); -} 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 deleted file mode 100644 index 7e8d306319172..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/TypeHintAware.java +++ /dev/null @@ -1,33 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.type.TypeHint; -import java.io.Serializable; - -/** @param */ -@Audience(Audience.Type.INTERNAL) -public interface TypeHintAware extends Serializable { - - /** - * Retrieve type hint associated with this object. Mostly this represents type returned by - * function. - * - * @return {@link TypeHint} associated with this object - */ - TypeHint getTypeHint(); -} 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 deleted file mode 100644 index 981b07639b9ee..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/UnaryPredicate.java +++ /dev/null @@ -1,23 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Binary function of X returning boolean. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface UnaryPredicate extends UnaryFunction {} 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 deleted file mode 100644 index 5ef98ab309169..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/functional/VoidFunction.java +++ /dev/null @@ -1,27 +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.core.client.functional; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** Function taking zero arguments. */ -@Audience(Audience.Type.CLIENT) -@FunctionalInterface -public interface VoidFunction extends Serializable { - - T apply(); -} 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/BoundedReader.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java deleted file mode 100644 index 6181e1828d034..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/BoundedReader.java +++ /dev/null @@ -1,27 +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.core.client.io; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** - * Reader of bounded data. - * - *

The reader can split the bounded data into smaller pieces. By default it is non splittable. - */ -@Audience(Audience.Type.CLIENT) -public interface BoundedReader extends CloseableIterator, 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/cz/seznam/euphoria/core/client/io/CloseableIterator.java deleted file mode 100644 index 81fbe8cc57987..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/CloseableIterator.java +++ /dev/null @@ -1,24 +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.core.client.io; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Closeable; -import java.util.Iterator; - -/** {@link Iterator} that is {@link Closeable}. */ -@Audience(Audience.Type.EXECUTOR) -public interface CloseableIterator extends Iterator, Closeable {} 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 deleted file mode 100644 index 38d6011a7dfce..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/Context.java +++ /dev/null @@ -1,22 +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.core.client.io; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Used in user defined functions to access environment methods. */ -@Audience(Audience.Type.CLIENT) -public interface Context extends Environment {} 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 deleted file mode 100644 index 923d329908fc8..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/io/ExternalIterable.java +++ /dev/null @@ -1,28 +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.core.client.io; - -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). */ -@Audience(Audience.Type.EXECUTOR) -public interface ExternalIterable extends Iterable, Closeable { - - // don't throw exceptions - @Override - void close(); -} 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/WindowAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java deleted file mode 100644 index 301619adc3ca9..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowAware.java +++ /dev/null @@ -1,32 +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.core.client.operator; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; - -/** - * Operator aware of windows. - * - * @param the type of elements processed - * @param the type of windows handled - */ -@Audience(Audience.Type.INTERNAL) -public interface WindowAware> { - - Windowing getWindowing(); -} 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 deleted file mode 100644 index 087182ff6f1f7..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowingRequiredException.java +++ /dev/null @@ -1,29 +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.core.client.operator; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** - * Thrown by executors at flow submission time when an invalid flow set up is detected, requiring - * the user to explicitly provide a windowing strategy to a certain operator. - */ -@Audience(Audience.Type.EXECUTOR) -public class WindowingRequiredException extends IllegalStateException { - public WindowingRequiredException(String message) { - super(message); - } -} 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 deleted file mode 100644 index eb1140fcb7268..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/ComputationHint.java +++ /dev/null @@ -1,26 +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.core.client.operator.hint; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Extra information for runner about Dataset computation. */ -@Audience(Audience.Type.CLIENT) -public enum ComputationHint implements OutputHint { - - /** Dataset is expensive to compute. */ - EXPENSIVE -} 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 deleted file mode 100644 index 6ccaaf0f9cc88..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/OutputHint.java +++ /dev/null @@ -1,23 +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.core.client.operator.hint; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** Specify hints about runtime data characteristics. */ -@Audience(Audience.Type.INTERNAL) -public interface OutputHint extends Serializable {} 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 deleted file mode 100644 index c128e34dae218..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/hint/SizeHint.java +++ /dev/null @@ -1,28 +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.core.client.operator.hint; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** Extra information for runner about Dataset size. */ -@Audience(Audience.Type.CLIENT) -public enum SizeHint implements OutputHint { - /** - * Indicate to runner that dataset can fit in memory and this information could be used for - * optimization (e.g. Broadcast hash join). - */ - FITS_IN_MEMORY -} 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/MergingStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java deleted file mode 100644 index 28dac4625743c..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/MergingStorageDescriptor.java +++ /dev/null @@ -1,31 +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.core.client.operator.state; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.BinaryFunction; - -/** - * Optional interface for descriptor to implement in order to enable state merging. - * - * @param the type of elements referred to through this descriptor - */ -@Audience(Audience.Type.CLIENT) -public interface MergingStorageDescriptor { - - /** @return the merging function for state storages. */ - BinaryFunction, ? extends Storage, Void> getMerger(); -} 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 deleted file mode 100644 index bbbfbd87dfd7f..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateContext.java +++ /dev/null @@ -1,31 +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.core.client.operator.state; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.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 { - - /** @return {@code StorageProvider} implementation. */ - StorageProvider getStorageProvider(); - - /** @return {@code SpillTools} for possible externalization of data to local storage. */ - SpillTools getSpillTools(); -} 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 deleted file mode 100644 index a4e082db77084..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/Storage.java +++ /dev/null @@ -1,26 +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.core.client.operator.state; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** A storage for state data. */ -@Audience(Audience.Type.INTERNAL) -public interface Storage { - - /** Clear the storage. */ - void clear(); -} 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 deleted file mode 100644 index 6a6c873fa424b..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StorageDescriptor.java +++ /dev/null @@ -1,35 +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.core.client.operator.state; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.io.Serializable; - -/** Descriptor of storage. */ -@Audience(Audience.Type.INTERNAL) -public abstract class StorageDescriptor implements Serializable { - - final String name; - - protected StorageDescriptor(String name) { - this.name = name; - } - - /** @return the name of the storage with scope of operator and key */ - public String getName() { - return name; - } -} 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 deleted file mode 100644 index 4aae519948d40..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ValueStorage.java +++ /dev/null @@ -1,37 +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.core.client.operator.state; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** - * State storage for single value. - * - * @param the type of value stored - */ -@Audience(Audience.Type.CLIENT) -public interface ValueStorage extends Storage { - - /** - * Set the value in this state. - * - * @param value the value to store - */ - void set(T value); - - /** @return the currently stored value */ - T get(); -} 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/Scheduler.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java deleted file mode 100644 index f3c8a1793a0da..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/Scheduler.java +++ /dev/null @@ -1,35 +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.core.time; - -import cz.seznam.euphoria.core.annotation.audience.Audience; -import java.time.Duration; - -/** TODO: complete javadoc. */ -@Audience(Audience.Type.EXECUTOR) -public interface Scheduler { - - /** - * Submit the given runnable for repeated execution. - * - * @param period the period at which to repeatedly invoke the given action - * @param r the action to invoke - */ - void schedulePeriodically(Duration period, Runnable r); - - /** Cancel all scheduled tasks. */ - void shutdown(); -} 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 deleted file mode 100644 index 4466ef2f54953..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/time/TimeProviderAware.java +++ /dev/null @@ -1,25 +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.core.time; - -import cz.seznam.euphoria.core.annotation.audience.Audience; - -/** TODO: complete javadoc. */ -@Audience(Audience.Type.INTERNAL) -public interface TimeProviderAware { - - TimeProvider getTimeProvider(); -} 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 68% 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 821b4a6b9fe4c..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 63% 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 afb9556260306..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 64% 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 fcb1fc9068939..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 67% 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 7514d7936a6bc..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 66% 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 42278046a65ea..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 68% 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 f943389f6abaf..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..4a5fbb9c78bdf --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Accumulator.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** + * Accumulators collect values from user functions. Accumulators allow user to calculate statistics + * during the flow execution. + * + *

Accumulators are inspired by the Hadoop/MapReduce counters. + */ +@Audience(Audience.Type.INTERNAL) +public interface Accumulator {} 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 70% 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 170b71595c645..01e8a9555f315 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.util.Settings; 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; /** * Provides access to an accumulator backend service. It is intended to be implemented by third diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..b5e93ef5d3b9d --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/accumulators/Counter.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.accumulators; + +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) +public interface Counter extends Accumulator { + + /** + * Increment counter by given value. + * + * @param value Value to be added to the counter. + */ + void increment(long value); + + /** Increment counter by one. */ + void increment(); +} 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 59% 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 f6b30bc5ecfc3..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 63% 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 02a7fb42a9d51..538b908fceaca 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 82% 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 7d591977887d1..2d760bf8a23ba 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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; 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 69% 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 c456a09a24961..22d858cebb1ab 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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.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; /** * 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 65% 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 77e58f53a9249..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 61% 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 201251030392a..49c147fcdf734 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +15,15 @@ * 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 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; /** {@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 63% 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 867033f0dbf3a..8e93eee230475 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +15,15 @@ * 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 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; /** {@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 62% 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 362dd3d99e342..b88c3bd74ded5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.CountTrigger; -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.triggers.CountTrigger; +import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; /** 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 69% 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 85ee164a6ee35..6581fab01b7e0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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; +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; /** Windowing with single window across the whole dataset. Suitable for batch processing. */ @Audience(Audience.Type.CLIENT) @@ -60,8 +62,9 @@ public int hashCode() { } /** Singleton window. */ - public static final class Window - extends cz.seznam.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/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 62% 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 82a4a488c0563..c17acb31dcbd1 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.util.Pair; 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; /** * @param 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 81% 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 14c4517983e02..168ef1d6517e6 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,17 +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 static com.google.common.base.Preconditions.checkArgument; -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 java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -33,6 +28,13 @@ import java.util.List; 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.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/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 75% 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 ae0493a2fcc6a..ce3758f33d4ba 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 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 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; /** 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 79% 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 d64c96942c029..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 80% 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 87fb464620867..eeb8cc2e1a1b9 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 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 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; /** Time sliding windowing. */ @Audience(Audience.Type.CLIENT) 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 67% 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 02a9fc6ffdce5..ac8952c173cb3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 59% 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 8f75af4e68f4c..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 65% 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 20f12d0686d97..5ad6df3b60200 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.triggers.Trigger; 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; /** A windowing policy of a dataset. All implementations must implement equals/hashCode. */ @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/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 88% 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 636a22621555f..cc002474c85db 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,8 @@ * 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 java.io.IOException; import java.io.ObjectOutputStream; import java.io.OutputStream; @@ -37,6 +31,14 @@ 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; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..774058abc78ef --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/flow/Util.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.flow; + +import javax.annotation.Nullable; + +class Util { + + private Util() {} + + /** + * Trim string (instead of empty string return null). + * + * @param s input string + * @return non-empty trimmed string or null + */ + @Nullable + static String trimToNull(@Nullable String s) { + if (s == null) { + return null; + } + s = s.trim(); + if (s.isEmpty()) { + return null; + } + return s; + } +} 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 64% 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 842ebd1b13c93..e641a009d46e7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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 new file mode 100644 index 0000000000000..3f1b28bb1a7c4 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/BinaryFunctor.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** Functor of two arguments. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface BinaryFunctor extends Serializable { + + void apply(LeftT left, RightT right, Collector context); +} diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..656b7dd389562 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableBinaryFunction.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Function of two arguments of the same type. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface CombinableBinaryFunction extends SameArgsBinaryFunction {} diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..41a6ae98f4d4b --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/CombinableReduceFunction.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +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 + * to be commutative associative. + */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface CombinableReduceFunction extends ReduceFunction {} 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 67% 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 dc71b5b6a6078..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/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 new file mode 100644 index 0000000000000..6d426ea5856b9 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/Consumer.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** A consumer of given type. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface Consumer extends Serializable { + + /** + * Consume given element. + * + * @param element the element to be consumed. + */ + void accept(T element); +} 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 new file mode 100644 index 0000000000000..8d21713eac9ee --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ExtractEventTime.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** @param */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface ExtractEventTime extends Serializable { + + /** + * Extracts event time (in millis since epoch) of the given element. + * + * @param elem input element + * @return the extracted event time + */ + long extractTimestamp(InputT elem); +} 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 new file mode 100644 index 0000000000000..f1ed706dd9359 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/ReduceFunction.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +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). + */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface ReduceFunction extends UnaryFunction, OutputT> {} 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 62% 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 79302da51c92e..c596bdb79e7a8 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/SameArgsBinaryFunction.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java new file mode 100644 index 0000000000000..adcdfbff3d74b --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/SameArgsBinaryFunction.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Function of two arguments of the same type. */ +@Audience(Audience.Type.INTERNAL) +@FunctionalInterface +public interface SameArgsBinaryFunction + extends BinaryFunction {} 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 new file mode 100644 index 0000000000000..e70009b2bbacb --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TernaryFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Function taking three arguments. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface TernaryFunction extends Serializable { + + OutputT apply(FirstT first, SecondT second, ThirdT third); +} 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 new file mode 100644 index 0000000000000..7fb831dd42904 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/TypeHintAware.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** @param */ +@Audience(Audience.Type.INTERNAL) +public interface TypeHintAware extends Serializable { + + /** + * Retrieve type hint associated with this object. Mostly this represents type returned by + * function. + * + * @return {@link TypeHint} associated with this object + */ + TypeHint getTypeHint(); +} 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 61% 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 a07b6efdfdc41..a45259a97ff4b 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 61% 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 0d543e88cc5c5..ae39e66349d59 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Context; 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; /** * Function of single argument with access to Euphoria environment via context. 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 57% 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 bc37cab39b0de..d239c9e2a8643 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.Collector; 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; /** * Functor of single argument. Functor can produce zero or more elements in return to a call, for diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..e2115ebc5dc4c --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/UnaryPredicate.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Binary function of X returning boolean. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface UnaryPredicate extends UnaryFunction {} 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 new file mode 100644 index 0000000000000..8e2a7c9d996a0 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/functional/VoidFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.functional; + +import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Function taking zero arguments. */ +@Audience(Audience.Type.CLIENT) +@FunctionalInterface +public interface VoidFunction extends Serializable { + + T apply(); +} 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 77% 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 015b3b0a943d7..34b5b8f44b55c 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 cz.seznam.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 new file mode 100644 index 0000000000000..eaf9d9b85c51f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/BoundedReader.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import java.io.Serializable; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** + * Reader of bounded data. + * + *

The reader can split the bounded data into smaller pieces. By default it is non splittable. + */ +@Audience(Audience.Type.CLIENT) +public interface BoundedReader extends CloseableIterator, Serializable {} 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 new file mode 100644 index 0000000000000..07a8c7a14f8eb --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/CloseableIterator.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.io; + +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) +public interface CloseableIterator extends Iterator, Closeable {} 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 64% 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 c7466b865bf34..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..8ea8801d8b45f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/Context.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.io; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Used in user defined functions to access environment methods. */ +@Audience(Audience.Type.CLIENT) +public interface Context extends Environment {} 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 76% 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 348b0cf0a56d8..01ae2789cb86a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.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/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 72% 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 47b5b6db90d7b..5de8bac9bdec3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 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; +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; /** Various {@link DataSink} related utilities. */ public class DataSinks { 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 72% 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 f9c26be1911b5..4c045929bfebe 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 61% 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 6342c2e323274..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/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 new file mode 100644 index 0000000000000..a6bcd85525e89 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/io/ExternalIterable.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.io; + +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) +public interface ExternalIterable extends Iterable, Closeable { + + // don't throw exceptions + @Override + void close(); +} 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 84% 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 e4b90c377468e..2d114cbd3560f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +15,8 @@ * 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 java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -27,6 +26,9 @@ 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/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 92% 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 e06b4a8371a94..a5eaa4d89fa33 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 com.google.common.collect.Lists; -import cz.seznam.euphoria.core.annotation.audience.Audience; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -29,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/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 88% 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 f7c6332aaa43b..e787b49c43950 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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.client.functional.UnaryFunction; -import cz.seznam.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/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 80% 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 ae44ef50cde7c..45b23341283f0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,17 +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 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 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/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 86% 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 f9b2565511150..311ea54e01295 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 68% 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 67b3738817a6f..00f9a1c599943 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 64% 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 2f6c6fac985a1..db9291e0d67cb 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 75% 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 208055a5fdd8e..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 62% 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 843c985a5c9e3..2a9d566bd3cb6 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -19,11 +21,11 @@ * 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 cz.seznam.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/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 65% 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 4dfbfe91e0dd6..3a531dbe0a501 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 75% 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 7b5f4e6213a7c..8fe6b789d3346 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 77% 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 37ed781aae2a8..a990aadfe020e 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 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; +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; /** * Composite operator using two {@link Filter} operators to split a {@link Dataset} into two subsets 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 78% 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 a070a32e99bcf..7f206343a6572 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +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.ExtractEventTime; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; 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; +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; /** * A convenient alias for assignment of event time. 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 72% 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 34a7f14a607ee..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 79% 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 539f8c66e65d8..3abdbe0544fce 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,23 +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.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 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; +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; /** * 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 81% 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 3eb9b76529637..1474d19d52955 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,25 +15,25 @@ * 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 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; +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; /** * 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 55% 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 c67cce57c07b8..f55d41049c1a6 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +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 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; +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; /** * Operator working element-wise, with no context between elements. No windowing scheme is needed to 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 79% 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 9ba213b2c81e3..64a1ead3bab88 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +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.UnaryPredicate; -import cz.seznam.euphoria.core.client.operator.hint.OutputHint; -import cz.seznam.euphoria.core.executor.graph.DAG; 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; +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; /** * Operator performing a filter operation. 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 86% 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 a8c55c3f46d19..14a274c3335af 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,22 +15,22 @@ * 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 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; +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; /** * 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 80% 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 27acec96ef496..5a872506d866e 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 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; +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; /** * Full outer join of two input datasets producing single new dataset. 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 89% 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 56e084c33b698..04e9672876443 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,35 +15,35 @@ * 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 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; +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; /** * 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 81% 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 cf06de0228586..4d44a97ddbc53 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 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; +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; /** * Left outer join of two input datasets producing single new dataset. 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 82% 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 bfd282359c1b9..2c14340bce6df 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,21 +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 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 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; +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; /** * Simple one-to-one transformation of input elements. It is a special case of {@link FlatMap} with 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 69% 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 fa3e2a2f5023f..1397420998366 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +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 static com.google.common.base.Preconditions.checkArgument; -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 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; /** An operator base class. All operators inherit his class. */ @Audience(Audience.Type.INTERNAL) 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 72% 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 d228873c4f1f9..64860ee5c563a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; 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; /** * 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/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 87% 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 bcc725d066e80..64bd988a2f0d5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,45 +15,45 @@ * 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 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; +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; /** * 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 88% 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 326822fa88a64..420e62e1b9d12 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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; +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; /** * A {@link ReduceStateByKey} operator is a stateful, complex, lower-level-api, but very powerful @@ -294,8 +297,8 @@ public static class DatasetBuilder4 the type of elements processed + * @param the type of windows handled + */ +@Audience(Audience.Type.INTERNAL) +public interface WindowAware> { + + Windowing getWindowing(); +} 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java similarity index 54% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/WindowWiseOperator.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowWiseOperator.java index ae321bf7f1c91..40a7a6be3560c 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/org/apache/beam/sdk/extensions/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 * @@ -13,13 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -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 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; /** Operator working on some context. */ @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/WindowingRequiredException.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowingRequiredException.java new file mode 100644 index 0000000000000..ecda91d02fd1f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowingRequiredException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** + * Thrown by executors at flow submission time when an invalid flow set up is detected, requiring + * the user to explicitly provide a windowing strategy to a certain operator. + */ +@Audience(Audience.Type.EXECUTOR) +public class WindowingRequiredException extends IllegalStateException { + public WindowingRequiredException(String message) { + super(message); + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java new file mode 100644 index 0000000000000..5a3f9006331bb --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/ComputationHint.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Extra information for runner about Dataset computation. */ +@Audience(Audience.Type.CLIENT) +public enum ComputationHint implements OutputHint { + + /** Dataset is expensive to compute. */ + EXPENSIVE +} 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 new file mode 100644 index 0000000000000..fdd0ccb42b79e --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/OutputHint.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint; + +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) +public interface OutputHint extends Serializable {} diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java new file mode 100644 index 0000000000000..576fe92c19470 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/SizeHint.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.hint; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** Extra information for runner about Dataset size. */ +@Audience(Audience.Type.CLIENT) +public enum SizeHint implements OutputHint { + /** + * Indicate to runner that dataset can fit in memory and this information could be used for + * optimization (e.g. Broadcast hash join). + */ + FITS_IN_MEMORY +} diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/package-info.java new file mode 100644 index 0000000000000..e9e89a1c032bd --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/hint/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.hint; diff --git a/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/package-info.java new file mode 100644 index 0000000000000..d7a3b1f0f16b5 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/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.client.operator; 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ListStorage.java similarity index 62% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorage.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ListStorage.java index 7604f412ad6f1..97458da58fdea 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/org/apache/beam/sdk/extensions/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 * @@ -13,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 storing lists. 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ListStorageDescriptor.java similarity index 60% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/ListStorageDescriptor.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ListStorageDescriptor.java index 9dc45a671887f..7fe4a2d181ed9 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/org/apache/beam/sdk/extensions/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 * @@ -13,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; /** Descriptor of list 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/operator/state/MergingStorageDescriptor.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/MergingStorageDescriptor.java new file mode 100644 index 0000000000000..e5b4f3480da66 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/MergingStorageDescriptor.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunction; + +/** + * Optional interface for descriptor to implement in order to enable state merging. + * + * @param the type of elements referred to through this descriptor + */ +@Audience(Audience.Type.CLIENT) +public interface MergingStorageDescriptor { + + /** @return the merging function for state storages. */ + BinaryFunction, ? extends Storage, Void> getMerger(); +} 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/State.java similarity index 62% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/State.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/State.java index f7478f62edf4e..31975d105adf9 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/org/apache/beam/sdk/extensions/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 * @@ -13,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.io.Collector; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; /** A state for stateful operations. */ @Audience(Audience.Type.CLIENT) 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 new file mode 100644 index 0000000000000..0a0cf4294a07f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateContext.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** A context that is passed to stateful operators when creating {@code State}. */ +@Audience(Audience.Type.CLIENT) +public interface StateContext extends Serializable { + + /** @return {@code StorageProvider} implementation. */ + StorageProvider getStorageProvider(); + + /** @return {@code SpillTools} for possible externalization of data to local storage. */ + SpillTools getSpillTools(); +} 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java similarity index 59% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateFactory.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateFactory.java index 665f5da498398..247608a9679cb 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/org/apache/beam/sdk/extensions/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 * @@ -13,13 +15,13 @@ * 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.annotation.stability.Experimental; -import cz.seznam.euphoria.core.client.io.Collector; 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; /** Factory for states. */ @Audience(Audience.Type.CLIENT) 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/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java similarity index 61% rename from sdks/java/extensions/euphoria/euphoria-core/src/main/java/cz/seznam/euphoria/core/client/operator/state/StateMerger.java rename to sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StateMerger.java index ce42561a41dbb..a757e416e90d8 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/org/apache/beam/sdk/extensions/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 * @@ -13,16 +15,16 @@ * 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 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 - * cz.seznam.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/Storage.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/Storage.java new file mode 100644 index 0000000000000..e7be21148d94d --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/Storage.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** A storage for state data. */ +@Audience(Audience.Type.INTERNAL) +public interface Storage { + + /** Clear the storage. */ + void clear(); +} 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 new file mode 100644 index 0000000000000..248db127dc2d3 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/StorageDescriptor.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** Descriptor of storage. */ +@Audience(Audience.Type.INTERNAL) +public abstract class StorageDescriptor implements Serializable { + + final String name; + + protected StorageDescriptor(String name) { + this.name = name; + } + + /** @return the name of the storage with scope of operator and key */ + public String getName() { + return name; + } +} 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 68% 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 9a5b97a33a08d..0b33632246d39 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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/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 new file mode 100644 index 0000000000000..bdb2a9dbcae2a --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/state/ValueStorage.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.client.operator.state; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** + * State storage for single value. + * + * @param the type of value stored + */ +@Audience(Audience.Type.CLIENT) +public interface ValueStorage extends Storage { + + /** + * Set the value in this state. + * + * @param value the value to store + */ + void set(T value); + + /** @return the currently stored value */ + T get(); +} 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 81% 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 c0c797264e150..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 74% 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 1dee73071ebb3..1c88c6de95dbb 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; 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; /** * Composite {@link Trigger} that fires once after at least one of sub-triggers have fired. In other 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 63% 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 62ef3ac2321fb..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 65% 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 e206f66a2612f..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 76% 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 b22f42f607431..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 72% 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 607f0bada3fc6..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 83% 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 d819d346fb23d..3a14eaf66a6fd 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +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 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; +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; /** * Trigger determines when a window result should be flushed. 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 65% 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 50d70ca374e94..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 61% 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 fc9a525633355..751f3f4a428ac 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 62% 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 4c5d45353e784..fabacb3c92512 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.client.functional.ReduceFunctor; -import cz.seznam.euphoria.core.client.io.Collector; 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; /** TODO: complete javadoc. */ public class TypeAwareReduceFunctor 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 60% 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 b1174044a039c..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 61% 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 5247ae580fe97..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 79% 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 c52b1de3158d1..06800eee14a54 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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 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/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 70% 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 e10fcb2f27cb0..e6d040644c8cb 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 static java.util.Objects.requireNonNull; -import cz.seznam.euphoria.core.annotation.audience.Audience; import javax.annotation.Nullable; +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/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 75% 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 965380ba4768c..68ff05153178a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 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; +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; /** Apply a folding function. */ public class Fold implements Serializable { 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 61% 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 9101407e93f17..44cfd03e35753 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 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; +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; /** Calculate maximum. */ @Audience(Audience.Type.CLIENT) 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 87% 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 f5d6ec8cb7bac..e066f261a5434 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.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/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 61% 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 772850d0849d8..46cf1d32e7009 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.functional.CombinableReduceFunction; 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; /** Provides commonly used function objects around computing sums. */ @Audience(Audience.Type.CLIENT) 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 72% 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 66299f82c2930..00191518f7074 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 69% 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 ae43837992355..020d3918f1edf 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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.client.flow.Flow; 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; 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 72% 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 2d4e8ff13d973..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 66% 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 9c47d4bab4277..291a6d08f0929 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +15,19 @@ * 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 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; +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; /** The client side, public, technology independent interface to an executor. */ @Audience(Audience.Type.CLIENT) 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 89% 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 046c71a1775da..6e290f14b2509 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,17 +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 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 java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -34,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/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 76% 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 290844e39454e..e0084aee1aaf9 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,25 +15,25 @@ * 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 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 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; /** 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 73% 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 ce45a9d2c40a1..57d55466c4189 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +15,11 @@ * 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 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/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 92% 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 7c54df3d69f57..4e01486ec18f7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +15,8 @@ * 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 cz.seznam.euphoria.core.client.util.Pair; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -32,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/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 83% 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 0cf38f25edefe..40d09137fb048 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +15,14 @@ * 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 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/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 80% 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 f40c6cb5aa395..9bfaf420989a5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,40 +15,40 @@ * 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 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 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; /** * An implementation of a RSBK group reducer of an ordered stream of already grouped (by a specific @@ -262,7 +264,8 @@ 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 82% 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 eeffaf768d035..77be84fc263de 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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 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/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 73% 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 4530bfcc94469..2b6ec3e215cc0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 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; +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; /** * Maintains triggers storages in memory for referencing through value descriptors in the scope of a 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 87% 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 aeccdaa6ddda3..b5ee255088725 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +15,8 @@ * 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 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; @@ -28,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/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 82% 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 d198dde85465d..92f216df89e44 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +15,9 @@ * 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 java.io.File; import java.util.ArrayList; import java.util.Collection; @@ -29,6 +26,11 @@ 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; 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 65% 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 65976c551e642..4fcbe8c45294c 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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 cz.seznam.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/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 72% 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 75e71ed50a3d8..c10e925eb6b6e 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +15,15 @@ * 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 cz.seznam.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/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 58% 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 395347aaa5c28..3cecc92ce7a74 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.io.ExternalIterable; 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; /** * An {@code ExternalIterable}, that is stored in memory. Use this class with care, because it might 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 69% 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 7e823f66b9a2d..1b6d492071f4f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +15,19 @@ * 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 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; +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; /** 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 55% 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 2f7f73f7bb37a..2cbfcb915bf54 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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; +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; /** Util class when specific executors use the same methods for operator translation. */ @Audience(Audience.Type.EXECUTOR) 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 70% 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 eef302f379ecd..679cfc2cfaa4b 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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; +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; /** * A {@code Context} that holds only single value. There is no window associated with the value, so 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/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 new file mode 100644 index 0000000000000..0ce4674588a42 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/Scheduler.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.time; + +import java.time.Duration; +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** TODO: complete javadoc. */ +@Audience(Audience.Type.EXECUTOR) +public interface Scheduler { + + /** + * Submit the given runnable for repeated execution. + * + * @param period the period at which to repeatedly invoke the given action + * @param r the action to invoke + */ + void schedulePeriodically(Duration period, Runnable r); + + /** Cancel all scheduled tasks. */ + void shutdown(); +} 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 69% 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 32352361bc062..6e79580612999 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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 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/TimeProviderAware.java b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviderAware.java new file mode 100644 index 0000000000000..4974c7d9dd842 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-core/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/time/TimeProviderAware.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.core.time; + +import org.apache.beam.sdk.extensions.euphoria.core.annotation.audience.Audience; + +/** TODO: complete javadoc. */ +@Audience(Audience.Type.INTERNAL) +public interface TimeProviderAware { + + TimeProvider getTimeProvider(); +} 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 87% 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 40ae1432028ab..93b66a26ec33f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +15,14 @@ * 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 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/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 69% 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 2d5860d4ab4ff..12ab1a2cf6e05 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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 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; 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 86% 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 e34d44fa7c4dc..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 77% 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 142ad4f9dccf6..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 73% 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 d027173036569..b0903474d743e 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 89% 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 b00977b3de821..f43d60bbd3216 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 cz.seznam.euphoria.core.annotation.audience.Audience; -import cz.seznam.euphoria.core.client.util.Pair; 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 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/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 74% 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 544f1080fc71e..46d0b1175ab90 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.util.Pair; import java.time.Duration; import java.util.Arrays; import java.util.Collection; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.junit.Test; /** TODO: complete javadoc. */ 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 75% 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 4197c7f1bfc36..0ee3c53b4efa7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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; import static org.junit.Assert.assertEquals; 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 72% 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 9f9db034102c0..6fb7809b0c142 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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; import static org.junit.Assert.assertEquals; 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 63% 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 08bb5446ca884..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 88% 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 036c561f1b0bf..61c1e4e6a24f7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.util.Pair; import java.time.Duration; import java.util.Arrays; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +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. */ 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 67% 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 2261961f818b7..200df92375356 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 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.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; 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 68% 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 b363c3bd9692f..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 67% 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 065d87effc479..e0158577b9847 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +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 static org.junit.Assert.assertEquals; 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; /** TODO: complete javadoc. */ 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 70% 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 60789554a077e..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 87% 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 4988d59bd8db1..839eaeafd2128 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 static org.junit.Assert.assertEquals; import static org.mockito.Mockito.times; 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 75% 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 56c3c4d3e7660..1c2c7b5a8b416 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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; @@ -21,13 +23,13 @@ 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.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.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; /** TODO: complete javadoc. */ 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 76% 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 f53b33c7b772c..ca00a5f434aad 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +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 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.util.Pair; 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; /** Test operator CountByKey. */ 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 76% 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 e2af78f60b8e2..bd84d4b945b7d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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.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; /** Test operator Distinct. */ 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 69% 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 e0238b6100db5..14a02fe199542 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +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 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.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.junit.Test; /** Test operator Filter. */ 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 81% 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 b9ad3ab0033f5..7557ad53b8404 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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.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; /** Test operator FlatMap. */ 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 68% 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 c49c745e7b44c..93ba6d03392b4 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,26 +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 static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; 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 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; +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; /** Test usage of hints in different operators. */ 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 90% 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 d88bde94319d7..2fabc7903f2d8 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,21 +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 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; 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; +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; /** Test operator Join. */ 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 80% 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 3ec66dd800775..301e7fa173152 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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; /** Test operator MapElement. */ 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 88% 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 77daa9936220a..2e05927aaa9b5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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; @@ -21,12 +23,12 @@ 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; 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; /** Test operator ReduceByKey. */ 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 77% 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 f646cedc3865c..09e0f47866937 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,23 +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 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; +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; /** Test operator ReduceStateByKey. */ 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 78% 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 b2af41e09aac5..c18fa02588d2b 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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.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; 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; +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; /** Test behavior of operator {@code ReduceWindow}. */ 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 75% 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 0b9e0586c93df..73069fc20d5de 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +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 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.util.Pair; 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; /** Test behavior of operator {@code SumByKey}. */ 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 78% 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 ee3df13f5f894..e784df75a595a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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; @@ -21,11 +23,11 @@ import static org.junit.Assert.assertTrue; 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; /** Test behavior of operator {@code TopPerKey}. */ 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 82% 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 1817825eb70e1..7dc9cdd11b130 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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; /** Test behavior of operator {@code Union}. */ 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 63% 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 377ebdd9436bb..5466025cce3da 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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; +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; /** 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/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 74% 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 e1383ccddaeba..8226044232c36 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +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 static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import cz.seznam.euphoria.core.util.IOUtils; import java.io.IOException; import java.util.Arrays; import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.core.util.IOUtils; import org.junit.Test; /** Test behavior of IOUtils. */ 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 78% 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 d1faab5e8652c..5fc251a97a8dc 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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 org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -21,26 +23,6 @@ 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 java.time.Duration; import java.util.Arrays; import java.util.Collection; @@ -49,6 +31,26 @@ 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; +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; 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 92% 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 33d48dfbc0ef4..1dce1d25c1cd8 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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 static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; 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 90% 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 3453b1682b158..f644c06e2ce93 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; 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 81% 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 e2aec8e84c243..0eb8cd7cc630f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 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.apache.beam.sdk.extensions.euphoria.core.client.io.ExternalIterable; import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 86% 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 8395218dbab25..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 68% 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 9bc3762a912da..1c6e26d99460e 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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.File; import java.io.IOException; 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 76% 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 031e1888b50bb..78786621f58f3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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 java.util.Arrays; import java.util.List; 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 85% 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 4035261da4e1f..b5dfdb000316c 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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; 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 70% 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 6365541ffb4d6..7232d7e964091 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 static org.junit.Assert.assertEquals; 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: - * - *

    - *
  • local - the local executor (suitable for unit tests) - *
  • flink-test - a flink executor for running on the local machine (suitable for unit tests) - *
  • flink - a flink executor capable of running in a distributed fashion - *
  • spark-test - a local spark executor for running on the local machine (suitable for unit - * tests) - *
  • spark - a spark executor capable of running in a distributed fashion - *
- * - * @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 04541f9bf83eb..0000000000000 Binary files a/sdks/java/extensions/euphoria/euphoria-examples/src/test/resources/avro/data.avro and /dev/null differ 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 deleted file mode 100644 index d1ba1c75a2820..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Dataset.java +++ /dev/null @@ -1,72 +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.fluent; - -import static java.util.Objects.requireNonNull; - -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; - -/** TODO: complete javadoc. */ -public class Dataset { - - private final cz.seznam.euphoria.core.client.dataset.Dataset wrap; - - Dataset(cz.seznam.euphoria.core.client.dataset.Dataset wrap) { - this.wrap = requireNonNull(wrap); - } - - public cz.seznam.euphoria.core.client.dataset.Dataset unwrap() { - return this.wrap; - } - - public Dataset apply( - UnaryFunction, Output> output) { - return new Dataset<>(requireNonNull(output.apply(this.wrap)).output()); - } - - public Dataset mapElements(UnaryFunction f) { - return new Dataset<>(MapElements.of(this.wrap).using(requireNonNull(f)).output()); - } - - public Dataset flatMap(UnaryFunctor f) { - return new Dataset<>(FlatMap.of(this.wrap).using(requireNonNull(f)).output()); - } - - public Dataset distinct() { - return new Dataset<>(Distinct.of(this.wrap).output()); - } - - public Dataset union(Dataset other) { - return new Dataset<>(Union.of(this.wrap, other.wrap).output()); - } - - public > Dataset persist(OutputT dst) { - this.wrap.persist(dst); - return this; - } - - public void execute(Executor exec) throws Exception { - exec.submit(this.wrap.getFlow()).get(); - } -} 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 deleted file mode 100644 index aec8991b76b40..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Flow.java +++ /dev/null @@ -1,42 +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.fluent; - -import static java.util.Objects.requireNonNull; - -import cz.seznam.euphoria.core.client.io.DataSource; -import cz.seznam.euphoria.core.util.Settings; - -/** TODO: complete javadoc. */ -public class Flow { - private final cz.seznam.euphoria.core.client.flow.Flow wrap; - - Flow(cz.seznam.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)); - } - - public static Flow create(String name, Settings settings) { - return new Flow(cz.seznam.euphoria.core.client.flow.Flow.create(name, settings)); - } - - public Dataset read(DataSource src) { - return new Dataset<>(wrap.createInput(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/cz/seznam/euphoria/fluent/Fluent.java deleted file mode 100644 index f779701684d11..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/cz/seznam/euphoria/fluent/Fluent.java +++ /dev/null @@ -1,34 +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.fluent; - -import cz.seznam.euphoria.core.util.Settings; - -/** Helper class providing convenient start points into the fluent api. */ -public class Fluent { - - public static Flow flow(String name) { - return Flow.create(name); - } - - 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) { - return new Dataset<>(xs); - } -} 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 new file mode 100644 index 0000000000000..83bb9bb73a817 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Dataset.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** TODO: complete javadoc. */ +public class Dataset { + + private final org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset wrap; + + Dataset(org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset wrap) { + this.wrap = requireNonNull(wrap); + } + + public org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset unwrap() { + return this.wrap; + } + + public Dataset apply( + UnaryFunction< + org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset, + Output> output) { + return new Dataset<>(requireNonNull(output.apply(this.wrap)).output()); + } + + public Dataset mapElements(UnaryFunction f) { + return new Dataset<>(MapElements.of(this.wrap).using(requireNonNull(f)).output()); + } + + public Dataset flatMap(UnaryFunctor f) { + return new Dataset<>(FlatMap.of(this.wrap).using(requireNonNull(f)).output()); + } + + public Dataset distinct() { + return new Dataset<>(Distinct.of(this.wrap).output()); + } + + public Dataset union(Dataset other) { + return new Dataset<>(Union.of(this.wrap, other.wrap).output()); + } + + public > Dataset persist(OutputT dst) { + this.wrap.persist(dst); + return this; + } + + public void execute(Executor exec) throws Exception { + exec.submit(this.wrap.getFlow()).get(); + } +} 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 new file mode 100644 index 0000000000000..3bad553228104 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Flow.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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; + +/** TODO: complete javadoc. */ +public class Flow { + private final org.apache.beam.sdk.extensions.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(org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow.create(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)); + } + + public Dataset read(DataSource src) { + return new Dataset<>(wrap.createInput(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 new file mode 100644 index 0000000000000..6110a5e380b91 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-fluent/src/main/java/org/apache/beam/sdk/extensions/euphoria/fluent/Fluent.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.fluent; + +import org.apache.beam.sdk.extensions.euphoria.core.util.Settings; + +/** Helper class providing convenient start points into the fluent api. */ +public class Fluent { + + public static Flow flow(String name) { + return Flow.create(name); + } + + 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) { + 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 66% 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 bb0f758286ddd..ba2402f6a95fc 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 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.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; /** Test behavior of Fluent API. */ 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/Collector.java b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java deleted file mode 100644 index 0c1b40f105c3a..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/Collector.java +++ /dev/null @@ -1,21 +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.executor.local; - -interface Collector { - - void collect(T elem); -} 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 deleted file mode 100644 index d58d7994253c6..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ExecPath.java +++ /dev/null @@ -1,39 +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.executor.local; - -import cz.seznam.euphoria.core.client.operator.Operator; -import cz.seznam.euphoria.core.executor.graph.DAG; - -/** A series of transformations with single output operator. */ -class ExecPath { - - /** A DAG of operators. */ - private final DAG> dag; - - private ExecPath(DAG> dag) { - this.dag = dag; - } - - /** Create new ExecPath. */ - static ExecPath of(DAG> dag) { - return new ExecPath(dag); - } - - public DAG> dag() { - return dag; - } -} 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 deleted file mode 100644 index acc05b94ab303..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/cz/seznam/euphoria/executor/local/ProcessingTimeTriggerScheduler.java +++ /dev/null @@ -1,25 +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.executor.local; - -/** Trigger scheduler based on real wall-clock time (processing time). */ -public class ProcessingTimeTriggerScheduler extends AbstractTriggerScheduler { - - @Override - public long getCurrentTimestamp() { - return System.currentTimeMillis(); - } -} 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 91% 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 b3a6ea8c49519..18fc39ab00295 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; 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 52% 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 ab9709e112e51..f2665c747fd5d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +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 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; +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; class AttachedWindowing> implements Windowing { diff --git a/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..679d6703457dc --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/Collector.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.executor.local; + +interface Collector { + + void collect(T elem); +} 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 82% 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 997d78fc5909a..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..6d7c2263f4597 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ExecPath.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.executor.local; + +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 { + + /** A DAG of operators. */ + private final DAG> dag; + + private ExecPath(DAG> dag) { + this.dag = dag; + } + + /** Create new ExecPath. */ + static ExecPath of(DAG> dag) { + return new ExecPath(dag); + } + + public DAG> dag() { + return dag; + } +} 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 73% 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 cbcac349d198c..8617ebad5f058 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,17 +15,17 @@ * 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.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; 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/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 69% 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 14fd036c8940d..729cd4437448f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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/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 90% 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 7d222f540e59a..2523ff7159cb0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,39 +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 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 java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -72,6 +44,36 @@ 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; +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; 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 62% 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 fb5ad98f8c40f..c26964aef0d6b 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 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/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 52% 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 ea49e418fdf49..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 64% 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 01798f0ab566e..6deeaf8ecba86 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,15 +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 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; +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; /** Provider of state storage for local executor. */ public class LocalStorageProvider implements StorageProvider { 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 64% 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 97af61cf91384..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..a2cc80a284c7c --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-local/src/main/java/org/apache/beam/sdk/extensions/euphoria/executor/local/ProcessingTimeTriggerScheduler.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.executor.local; + +/** Trigger scheduler based on real wall-clock time (processing time). */ +public class ProcessingTimeTriggerScheduler extends AbstractTriggerScheduler { + + @Override + public long getCurrentTimestamp() { + return System.currentTimeMillis(); + } +} 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 91% 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 677bf80d9585b..88d092c5a4ed5 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,36 +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 static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toSet; -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; @@ -54,6 +32,30 @@ 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; +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; 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 78% 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 34e777f1c243a..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 65% 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 487ba96703d35..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 72% 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 d278ba07de90b..b961b08d52716 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.ScheduledExecutorService; 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 88% 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 c7ef0d47cc876..bc1798cef0436 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 com.google.common.collect.Iterables; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.util.Pair; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -27,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/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 60% 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 6401f14aea879..d77c70b84f93d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,21 +15,21 @@ * 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.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; +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; 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; @@ -35,7 +37,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/testkit/LocalOperatorTest.java b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java deleted file mode 100644 index 538faf79cb6f3..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-local/src/test/java/cz/seznam/euphoria/executor/local/testkit/LocalOperatorTest.java +++ /dev/null @@ -1,23 +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.executor.local.testkit; - -import cz.seznam.euphoria.operator.test.AllOperatorsSuite; - -/** - * Local operator test suite. - */ -public class LocalOperatorTest extends AllOperatorsSuite implements LocalExecutorProvider {} 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 88% 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 90f0d50f95f61..2d40a13dd3926 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,33 +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 java.util.Arrays.asList; import static java.util.stream.Collectors.toList; import static org.junit.Assert.assertTrue; 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 java.time.Duration; import java.util.HashSet; import java.util.List; @@ -48,6 +30,26 @@ 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; +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; /** Test basic operator functionality and ability to compile. */ 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 78% 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 24594357ff989..7b4db6b9c137d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,28 +15,28 @@ * 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 java.util.Arrays.asList; -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.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; 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 85% 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 066dce96f0043..cf1c79a7b36aa 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -23,38 +25,6 @@ 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 java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -68,6 +38,38 @@ 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; +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; 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 64% 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 3f55d5791dc9c..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 67% 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 2938cc9fdef59..5fd22644bc3d8 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 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; /** Test vector clocks. */ 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 84% 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 08beeb432085f..f4a6f5da0ecf7 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,37 +15,37 @@ * 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 java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; 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 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; +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; 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 51% 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 e8192f7504da5..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..ce3c7a0dbb224 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-local/src/test/java/org/apache/beam/sdk/extensions/euphoria/executor/local/testkit/LocalOperatorTest.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.executor.local.testkit; + +import org.apache.beam.sdk.extensions.euphoria.operator.test.AllOperatorsSuite; + +/** + * Local operator test suite. + */ +public class LocalOperatorTest extends AllOperatorsSuite implements LocalExecutorProvider {} 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/accumulators/SnapshotProvider.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java deleted file mode 100644 index c418abd1445e7..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/SnapshotProvider.java +++ /dev/null @@ -1,31 +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.operator.test.accumulators; - -import java.time.Duration; -import java.util.Map; - -/** - * Provider of snapshots. - */ -public interface SnapshotProvider { - - Map getCounterSnapshots(); - - Map> getHistogramSnapshots(); - - Map> getTimerSnapshots(); -} 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 deleted file mode 100644 index b492f5f8e91b4..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/accumulators/Snapshotable.java +++ /dev/null @@ -1,20 +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.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/junit/ExecutorEnvironment.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java deleted file mode 100644 index 2dabac6899f1a..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorEnvironment.java +++ /dev/null @@ -1,28 +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.operator.test.junit; - -import cz.seznam.euphoria.core.executor.Executor; - -/** - * Source of {@link Executor} with ability to shutdown it. - */ -public interface ExecutorEnvironment { - - Executor getExecutor(); - - void shutdown() throws Exception; -} 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 deleted file mode 100644 index 16b48ee886b67..0000000000000 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/cz/seznam/euphoria/operator/test/junit/ExecutorProvider.java +++ /dev/null @@ -1,24 +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.operator.test.junit; - -/** - * Provider of {@lin ExecutorEnvironment}. - */ -public interface ExecutorProvider { - - ExecutorEnvironment newExecutorEnvironment() throws Exception; -} 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 61% 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 76a8a8bce8c50..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 80% 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 d81c4c1749421..4ca7c4aaf177f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 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; +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; /** 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 76% 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 df0558525e561..933f02b0f7c35 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,19 +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 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.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; /** Test operator {@code CountByKey}. */ 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 75% 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 86d20e0a478f6..8f64c43eb4e84 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +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 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; 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; +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; /** Test for the {@link Distinct} operator. */ 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 59% 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 ff6f07946b3e7..974c1df67dc52 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,14 +15,14 @@ * 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.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.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; /** Test operator {@code Filter}. */ 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 72% 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 5256bb2be8e54..24f224085d09c 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +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 org.junit.Assert.assertEquals; -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.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; /** Test operator {@code FlatMap}. */ 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 62% 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 fbf830475a9e6..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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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 88% 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 e8ed22e00cd0b..e05203fe0b668 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,39 +15,39 @@ * 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 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.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; /** Test operator {@code Join}. */ 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 79% 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 5dc87194b93ab..2e8bc5157b2f2 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,25 +15,25 @@ * 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.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.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; 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 71% 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 41f8bfdb85b79..9fead76103157 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +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 org.junit.Assert.assertEquals; -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.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; /** Tests for operator {@code MapElements}. */ 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 90% 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 7408d153a6cb3..e23ceb6d25780 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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; @@ -21,39 +23,6 @@ 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 java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -66,6 +35,39 @@ 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; +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; /** Test operator {@code ReduceByKey}. */ 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 89% 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 d8b38f09bfe2e..fa01abca544b8 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,41 +15,12 @@ * 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.assertTrue; 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 java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -56,6 +29,35 @@ 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; +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; /** Test operator {@code ReduceStateByKey}. */ 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 69% 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 09d66960589ca..7b9a3d66fc76f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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.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.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; /** Test operator {@code ReduceByKey}. */ 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 70% 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 484b9f4181e8f..f0fe8693f8a40 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,26 +15,26 @@ * 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 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.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; /** Test that a sub-flow applied on sink is correctly preserved. */ 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 58% 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 b7e322e9d7628..a18aa1af4999a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,17 +15,17 @@ * 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.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.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; /** Test operator {@code SumByKey}. */ 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 75% 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 6008de4cd2c1d..97756244d7d7a 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,18 +15,18 @@ * 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 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.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; /** 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 82% 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 9c81886f50a4b..18c06d3f830cf 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +15,16 @@ * 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 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; +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; /** Test for operator {@code Union}. */ 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 55% 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 8c739f98cd4ed..2292d9f85b860 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,16 +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 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; 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/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 67% 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 3651f4ddf66ca..b74257b1ef0d3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,22 +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 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.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; /** 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 86% 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 8c95267600b7e..c8fd5b5e5fa5d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,38 +15,11 @@ * 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.fail; -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; @@ -52,6 +27,33 @@ 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; +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; /** Tests capabilities of {@link Windowing}. */ 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 60% 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 e9e3cd6f3a295..0f04d931c3af9 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,10 +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 cz.seznam.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/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 62% 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 a9b5f55fd9484..9eb809a7041c0 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,12 +15,12 @@ * 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 cz.seznam.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/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 66% 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 3a8955e8ae113..13919e0561061 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +15,13 @@ * 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 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/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 79% 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 8365d980e4abc..e66f81496d08d 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,20 +15,20 @@ * 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 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/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 new file mode 100644 index 0000000000000..ce10795cd129d --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/SnapshotProvider.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators; + +import java.time.Duration; +import java.util.Map; + +/** + * Provider of snapshots. + */ +public interface SnapshotProvider { + + Map getCounterSnapshots(); + + Map> getHistogramSnapshots(); + + Map> getTimerSnapshots(); +} diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..60e3db3a4b687 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/accumulators/Snapshotable.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 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 85% 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 61711f2191edb..a4dba44914a57 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,27 +15,28 @@ * 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.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; 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; +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; /** * 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/ExecutorEnvironment.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorEnvironment.java new file mode 100644 index 0000000000000..33fc3bb2471e0 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorEnvironment.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.operator.test.junit; + +import org.apache.beam.sdk.extensions.euphoria.core.executor.Executor; + +/** + * Source of {@link Executor} with ability to shutdown it. + */ +public interface ExecutorEnvironment { + + Executor getExecutor(); + + void shutdown() throws Exception; +} diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/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 new file mode 100644 index 0000000000000..510ba81d3ee04 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/junit/ExecutorProvider.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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 org.apache.beam.sdk.extensions.euphoria.operator.test.junit; + +/** + * Provider of {@lin ExecutorEnvironment}. + */ +public interface ExecutorProvider { + + ExecutorEnvironment newExecutorEnvironment() throws Exception; +} 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 94% 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 661042990946a..5f9596d44e20f 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,11 +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 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; @@ -26,6 +27,7 @@ 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; 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 75% 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 ecf4d0ae57bbd..e68543439ebf3 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,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; import com.google.common.collect.Lists; import java.lang.annotation.Inherited; 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/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/org/apache/beam/sdk/extensions/euphoria/testing/AbstractFlowTest.java similarity index 62% 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 43a48d9d80575..65df5a7608ead 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,13 +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 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; +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; /** * Abstract test class for user's {@link Flow} testing. 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 76% 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 cba2bb16853cc..a57cc184ce062 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 @@ -1,9 +1,11 @@ /* - * Copyright 2016-2018 Seznam.cz, a.s. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -13,7 +15,7 @@ * 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 java.util.Arrays; import java.util.List; 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;