Skip to content

Commit

Permalink
Move ActiveWindowSet and implementations to runners-core
Browse files Browse the repository at this point in the history
  • Loading branch information
kennknowles committed Dec 13, 2016
1 parent 91cc606 commit 9a71e6e
Show file tree
Hide file tree
Showing 8 changed files with 9 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.util;
package org.apache.beam.runners.core;

import com.google.common.annotations.VisibleForTesting;
import java.util.Collection;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.util;
package org.apache.beam.runners.core;

import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.util;
package org.apache.beam.runners.core;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.SideInputReader;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.TimerInternals;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,6 @@
import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.MergingActiveWindowSet;
import org.apache.beam.sdk.util.NonMergingActiveWindowSet;
import org.apache.beam.sdk.util.SideInputReader;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.TimerInternals;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,12 @@
import java.util.Collection;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.runners.core.ActiveWindowSet;
import org.apache.beam.runners.core.triggers.TriggerStateMachine.MergingTriggerInfo;
import org.apache.beam.runners.core.triggers.TriggerStateMachine.TriggerInfo;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.Timers;
import org.apache.beam.sdk.util.state.MergingStateAccessor;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.util;
package org.apache.beam.runners.core;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,14 +32,14 @@
import java.util.Map;
import java.util.Set;
import javax.annotation.Nullable;
import org.apache.beam.runners.core.ActiveWindowSet;
import org.apache.beam.runners.core.ActiveWindowSet.MergeCallback;
import org.apache.beam.runners.core.MergingActiveWindowSet;
import org.apache.beam.runners.core.NonMergingActiveWindowSet;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.ActiveWindowSet.MergeCallback;
import org.apache.beam.sdk.util.MergingActiveWindowSet;
import org.apache.beam.sdk.util.NonMergingActiveWindowSet;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.TimerInternals.TimerData;
import org.apache.beam.sdk.util.Timers;
Expand Down

0 comments on commit 9a71e6e

Please sign in to comment.