-
Notifications
You must be signed in to change notification settings - Fork 4.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[YAML] Allow windowing to be done in Java or Python. #30055
Merged
Merged
Changes from 3 commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
127 changes: 127 additions & 0 deletions
127
...vice/src/main/java/org/apache/beam/sdk/expansion/service/WindowIntoTransformProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,127 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this 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.expansion.service; | ||
|
||
import com.google.auto.service.AutoService; | ||
import com.google.auto.value.AutoValue; | ||
import java.io.IOException; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import org.apache.beam.model.pipeline.v1.RunnerApi; | ||
import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; | ||
import org.apache.beam.sdk.schemas.AutoValueSchema; | ||
import org.apache.beam.sdk.schemas.annotations.DefaultSchema; | ||
import org.apache.beam.sdk.schemas.transforms.SchemaTransform; | ||
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; | ||
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; | ||
import org.apache.beam.sdk.transforms.windowing.Window; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.PCollectionRowTuple; | ||
import org.apache.beam.sdk.values.Row; | ||
import org.apache.beam.sdk.values.WindowingStrategy; | ||
|
||
/** | ||
* An implementation of {@link TypedSchemaTransformProvider} for WindowInto. | ||
* | ||
* <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We | ||
* provide no backwards compatibility guarantees, and it should not be implemented outside the Beam | ||
* repository. | ||
*/ | ||
@SuppressWarnings({ | ||
"nullness" // TODO(https://github.com/apache/beam/issues/20497) | ||
}) | ||
@AutoService(SchemaTransformProvider.class) | ||
public class WindowIntoTransformProvider | ||
extends TypedSchemaTransformProvider<WindowIntoTransformProvider.Configuration> { | ||
private static final String INPUT_ROWS_TAG = "input"; | ||
|
||
protected static final String OUTPUT_ROWS_TAG = "output"; | ||
|
||
@Override | ||
protected Class<Configuration> configurationClass() { | ||
return Configuration.class; | ||
} | ||
|
||
@Override | ||
protected SchemaTransform from(Configuration configuration) { | ||
try { | ||
return new WindowIntoStrategy( | ||
(WindowingStrategy<Row, ?>) | ||
WindowingStrategyTranslation.fromProto( | ||
RunnerApi.WindowingStrategy.parseFrom( | ||
configuration.getSerializedWindowingStrategy()), | ||
null)); | ||
} catch (IOException exn) { | ||
throw new RuntimeException(exn); | ||
} | ||
} | ||
|
||
@Override | ||
public String identifier() { | ||
return "beam:schematransform:org.apache.beam:yaml:window_into_strategy:v1"; | ||
} | ||
|
||
@Override | ||
public List<String> inputCollectionNames() { | ||
return Collections.singletonList(INPUT_ROWS_TAG); | ||
} | ||
|
||
@Override | ||
public List<String> outputCollectionNames() { | ||
return Collections.singletonList(OUTPUT_ROWS_TAG); | ||
} | ||
|
||
@DefaultSchema(AutoValueSchema.class) | ||
@AutoValue | ||
public abstract static class Configuration { | ||
|
||
@SuppressWarnings({"AutoValueMutable", "mutable"}) | ||
public abstract byte[] getSerializedWindowingStrategy(); | ||
|
||
public static Builder builder() { | ||
return new AutoValue_WindowIntoTransformProvider_Configuration.Builder(); | ||
} | ||
|
||
@AutoValue.Builder | ||
public abstract static class Builder { | ||
|
||
public abstract Builder setSerializedWindowingStrategy(byte[] serializedWindowingStrategy); | ||
|
||
public abstract Configuration build(); | ||
} | ||
} | ||
|
||
private static class WindowIntoStrategy extends SchemaTransform { | ||
|
||
private final WindowingStrategy<Row, ?> windowingStrategy; | ||
|
||
WindowIntoStrategy(WindowingStrategy<Row, ?> windowingStrategy) { | ||
this.windowingStrategy = windowingStrategy; | ||
} | ||
|
||
@Override | ||
public PCollectionRowTuple expand(PCollectionRowTuple inputTuple) { | ||
PCollection<Row> input = inputTuple.get(INPUT_ROWS_TAG); | ||
return PCollectionRowTuple.of( | ||
OUTPUT_ROWS_TAG, | ||
input | ||
.apply(Window.Assign.<Row>createInternal(windowingStrategy)) | ||
.setCoder(input.getCoder())); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does there need to be a new provider? Seems as though
RenamingProvider
has all the relevant functionality and would allow mappings and defaults should there be Java built-in transforms that expose those in the futureThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This does a deeper rewriting of the arguments than renaming, though I was thinking that the renaming transform could be defined as a special case of this one. (The SQL-based one possibly as well.)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah I think they could be merged in either direction, but not a blocker. LGTM