Skip to content
Open
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -53,5 +53,7 @@ public interface OutputBuilder<T> extends WindowedValue<T> {

OutputBuilder<T> setOpenTelemetryContext(@Nullable Context openTelemetryContext);

OutputBuilder<T> setValueKind(ValueKind valueKind);

void output();
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ public T getValue() {

public abstract @Nullable Context getOpenTelemetryContext();

public abstract ValueKind getValueKind();

// todo #33176 specify additional metadata in the future
public static <T> ValueInSingleWindow<T> of(
T value,
Expand All @@ -81,6 +83,28 @@ public static <T> ValueInSingleWindow<T> of(
@Nullable Long currentRecordOffset,
CausedByDrain causedByDrain,
@Nullable Context openTelemetryContext) {
return of(
value,
timestamp,
window,
paneInfo,
currentRecordId,
currentRecordOffset,
causedByDrain,
openTelemetryContext,
ValueKind.INSERT);
}

public static <T> ValueInSingleWindow<T> of(
T value,
Instant timestamp,
BoundedWindow window,
PaneInfo paneInfo,
@Nullable String currentRecordId,
@Nullable Long currentRecordOffset,
CausedByDrain causedByDrain,
@Nullable Context openTelemetryContext,
ValueKind valueKind) {
return new AutoValue_ValueInSingleWindow<>(
value,
timestamp,
Expand All @@ -89,12 +113,22 @@ public static <T> ValueInSingleWindow<T> of(
currentRecordId,
currentRecordOffset,
causedByDrain,
openTelemetryContext);
openTelemetryContext,
valueKind);
}

public static <T> ValueInSingleWindow<T> of(
T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) {
return of(value, timestamp, window, paneInfo, null, null, CausedByDrain.NORMAL, null);
return of(
value,
timestamp,
window,
paneInfo,
null,
null,
CausedByDrain.NORMAL,
null,
ValueKind.INSERT);
}

/** A coder for {@link ValueInSingleWindow}. */
Expand Down Expand Up @@ -142,9 +176,9 @@ public void encode(
io.opentelemetry.context.Context openTelemetryContext =
windowedElem.getOpenTelemetryContext();
if (openTelemetryContext != null) {

OpenTelemetryContextPropagator.set(openTelemetryContext, builder);
}
builder.setValueKind(ValueKindUtil.toProto(windowedElem.getValueKind()));
BeamFnApi.Elements.ElementMetadata metadata = builder.build();
ByteArrayCoder.of().encode(metadata.toByteArray(), outStream);
}
Expand All @@ -166,6 +200,7 @@ public ValueInSingleWindow<T> decode(InputStream inStream, Coder.Context context
PaneInfo paneInfo = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream);
CausedByDrain causedByDrain = CausedByDrain.NORMAL;
io.opentelemetry.context.@Nullable Context openTelemetryContext = null;
ValueKind valueKind = ValueKind.INSERT;
if (WindowedValues.WindowedValueCoder.isMetadataSupported() && paneInfo.isElementMetadata()) {
BeamFnApi.Elements.ElementMetadata elementMetadata =
BeamFnApi.Elements.ElementMetadata.parseFrom(ByteArrayCoder.of().decode(inStream));
Expand All @@ -174,12 +209,21 @@ public ValueInSingleWindow<T> decode(InputStream inStream, Coder.Context context
? CausedByDrain.CAUSED_BY_DRAIN
: CausedByDrain.NORMAL;
openTelemetryContext = OpenTelemetryContextPropagator.read(elementMetadata);
valueKind = ValueKindUtil.fromProto(elementMetadata.getValueKind());
}

T value = valueCoder.decode(inStream, context);
// todo #33176 specify additional metadata in the future
return new AutoValue_ValueInSingleWindow<>(
value, timestamp, window, paneInfo, null, null, causedByDrain, openTelemetryContext);
value,
timestamp,
window,
paneInfo,
null,
null,
causedByDrain,
openTelemetryContext,
valueKind);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this 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.values;

/** The type of change operation represented by a Change Data Capture (CDC) record. */
public enum ValueKind {
/** Indicates a new record was created in the source system. */
INSERT,

/**
* Indicates the state of a record immediately <b>before</b> an update occurred. This is typically
* used to identify the previous values of modified columns or to locate the record via its
* primary key.
*/
UPDATE_BEFORE,

/**
* Indicates the state of a record immediately <b>after</b> an update occurred. Represents the
* current, valid state of the record following the change.
*/
UPDATE_AFTER,

/** Indicates that an existing record was removed from the source system. */
DELETE
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this 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.values;

import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;

/** Utility class for converting between {@link ValueKind} and {@link Elements.ValueKind.Enum}. */
public class ValueKindUtil {
Comment thread
ahmedabu98 marked this conversation as resolved.
Outdated
public static Elements.ValueKind.Enum toProto(ValueKind valueKind) {
switch (valueKind) {
case INSERT:
return Elements.ValueKind.Enum.INSERT;
case UPDATE_BEFORE:
return Elements.ValueKind.Enum.UPDATE_BEFORE;
case UPDATE_AFTER:
return Elements.ValueKind.Enum.UPDATE_AFTER;
case DELETE:
return Elements.ValueKind.Enum.DELETE;
default:
throw new IllegalArgumentException("Unknown ValueKind: " + valueKind);
}
}

public static ValueKind fromProto(Elements.ValueKind.Enum proto) {
switch (proto) {
case VALUE_KIND_UNSPECIFIED:
case INSERT:
return ValueKind.INSERT;
case UPDATE_BEFORE:
return ValueKind.UPDATE_BEFORE;
case UPDATE_AFTER:
return ValueKind.UPDATE_AFTER;
case DELETE:
return ValueKind.DELETE;
default:
throw new IllegalArgumentException("Unknown ValueKind: " + proto);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,10 @@ public interface WindowedValue<T> {

CausedByDrain causedByDrain();

/** Returns the {@link ValueKind} associated with this WindowedValue. */
@Pure
ValueKind getValueKind();

/**
* A representation of each of the actual values represented by this compressed {@link
* WindowedValue}, one per window.
Expand Down
Loading
Loading