|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.streaming.runtime.operators.windowing; |
| 20 | + |
| 21 | +import org.apache.flink.annotation.Internal; |
| 22 | +import org.apache.flink.annotation.VisibleForTesting; |
| 23 | +import org.apache.flink.api.common.state.State; |
| 24 | +import org.apache.flink.api.common.state.StateDescriptor; |
| 25 | +import org.apache.flink.api.common.state.v2.StateFuture; |
| 26 | +import org.apache.flink.core.state.StateFutureUtils; |
| 27 | +import org.apache.flink.metrics.MetricGroup; |
| 28 | +import org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncCountTrigger; |
| 29 | +import org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncEventTimeTrigger; |
| 30 | +import org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncProcessingTimeTrigger; |
| 31 | +import org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncPurgingTrigger; |
| 32 | +import org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncTrigger; |
| 33 | +import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows; |
| 34 | +import org.apache.flink.streaming.api.windowing.triggers.CountTrigger; |
| 35 | +import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger; |
| 36 | +import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger; |
| 37 | +import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger; |
| 38 | +import org.apache.flink.streaming.api.windowing.triggers.Trigger; |
| 39 | +import org.apache.flink.streaming.api.windowing.triggers.TriggerResult; |
| 40 | +import org.apache.flink.streaming.api.windowing.windows.Window; |
| 41 | + |
| 42 | +import javax.annotation.Nonnull; |
| 43 | + |
| 44 | +/** |
| 45 | + * A converter from {@code Trigger} to {@code AsyncTrigger}. |
| 46 | + * |
| 47 | + * <p>Basic triggers (e.g., {@code CountTrigger}) are directly converted to their async version. |
| 48 | + * |
| 49 | + * <p>Async-support triggers which implement {@code AsyncTriggerConvertable} (e.g., {@code |
| 50 | + * ProcessingTimeoutTrigger}) will use self-defined async version. |
| 51 | + * |
| 52 | + * <p>Other triggers are wrapped as an {@code AsyncTrigger}, whose internal functions are executed |
| 53 | + * in sync mode. |
| 54 | + */ |
| 55 | +@Internal |
| 56 | +public interface AsyncTriggerConverter { |
| 57 | + |
| 58 | + /** |
| 59 | + * Convert to an {@code AsyncTrigger}. The default implementation is only a wrapper of the |
| 60 | + * trigger, whose behaviours are all sync. |
| 61 | + * |
| 62 | + * <p>TODO: Return {@code AsyncTrigger} if {@code AsyncTrigger} becomes @PublicEvolving. |
| 63 | + * |
| 64 | + * @return The {@code AsyncTrigger} for async state processing. |
| 65 | + */ |
| 66 | + @Nonnull |
| 67 | + default Object convertToAsync() { |
| 68 | + return UserDefinedAsyncTrigger.of((Trigger<?, ?>) AsyncTriggerConverter.this); |
| 69 | + } |
| 70 | + |
| 71 | + @SuppressWarnings("unchecked") |
| 72 | + static <T, W extends Window> AsyncTrigger<T, W> convertToAsync(Trigger<T, W> trigger) { |
| 73 | + if (trigger instanceof CountTrigger) { |
| 74 | + return (AsyncTrigger<T, W>) |
| 75 | + AsyncCountTrigger.of(((CountTrigger<?>) trigger).getMaxCount()); |
| 76 | + } else if (trigger instanceof EventTimeTrigger) { |
| 77 | + return (AsyncTrigger<T, W>) AsyncEventTimeTrigger.create(); |
| 78 | + } else if (trigger instanceof ProcessingTimeTrigger) { |
| 79 | + return (AsyncTrigger<T, W>) AsyncProcessingTimeTrigger.create(); |
| 80 | + } else if (trigger instanceof PurgingTrigger) { |
| 81 | + return (AsyncTrigger<T, W>) |
| 82 | + AsyncPurgingTrigger.of( |
| 83 | + convertToAsync(((PurgingTrigger<?, ?>) trigger).getNestedTrigger())); |
| 84 | + } else if (trigger instanceof AsyncTriggerConverter) { |
| 85 | + return (AsyncTrigger<T, W>) ((AsyncTriggerConverter) trigger).convertToAsync(); |
| 86 | + } else { |
| 87 | + return UserDefinedAsyncTrigger.of(trigger); |
| 88 | + } |
| 89 | + } |
| 90 | + |
| 91 | + /** Convert non-support user-defined trigger to {@code AsyncTrigger}. */ |
| 92 | + class UserDefinedAsyncTrigger<T, W extends Window> extends AsyncTrigger<T, W> { |
| 93 | + private final Trigger<T, W> userDefinedTrigger; |
| 94 | + |
| 95 | + private UserDefinedAsyncTrigger(Trigger<T, W> userDefinedTrigger) { |
| 96 | + this.userDefinedTrigger = userDefinedTrigger; |
| 97 | + } |
| 98 | + |
| 99 | + @Override |
| 100 | + public StateFuture<TriggerResult> onElement( |
| 101 | + T element, long timestamp, W window, TriggerContext ctx) throws Exception { |
| 102 | + return StateFutureUtils.completedFuture( |
| 103 | + userDefinedTrigger.onElement( |
| 104 | + element, timestamp, window, AsyncTriggerContextConvertor.of(ctx))); |
| 105 | + } |
| 106 | + |
| 107 | + @Override |
| 108 | + public StateFuture<TriggerResult> onProcessingTime(long time, W window, TriggerContext ctx) |
| 109 | + throws Exception { |
| 110 | + return StateFutureUtils.completedFuture( |
| 111 | + userDefinedTrigger.onProcessingTime( |
| 112 | + time, window, AsyncTriggerContextConvertor.of(ctx))); |
| 113 | + } |
| 114 | + |
| 115 | + @Override |
| 116 | + public StateFuture<TriggerResult> onEventTime(long time, W window, TriggerContext ctx) |
| 117 | + throws Exception { |
| 118 | + return StateFutureUtils.completedFuture( |
| 119 | + userDefinedTrigger.onEventTime( |
| 120 | + time, window, AsyncTriggerContextConvertor.of(ctx))); |
| 121 | + } |
| 122 | + |
| 123 | + @Override |
| 124 | + public StateFuture<Void> clear(W window, TriggerContext ctx) throws Exception { |
| 125 | + userDefinedTrigger.clear(window, AsyncTriggerContextConvertor.of(ctx)); |
| 126 | + return StateFutureUtils.completedVoidFuture(); |
| 127 | + } |
| 128 | + |
| 129 | + @Override |
| 130 | + public boolean isEndOfStreamTrigger() { |
| 131 | + return userDefinedTrigger instanceof GlobalWindows.EndOfStreamTrigger; |
| 132 | + } |
| 133 | + |
| 134 | + public static <T, W extends Window> AsyncTrigger<T, W> of( |
| 135 | + Trigger<T, W> userDefinedTrigger) { |
| 136 | + return new UserDefinedAsyncTrigger<>(userDefinedTrigger); |
| 137 | + } |
| 138 | + |
| 139 | + /** |
| 140 | + * A converter from {@link AsyncTrigger.TriggerContext} to {@link Trigger.TriggerContext}. |
| 141 | + */ |
| 142 | + private static class AsyncTriggerContextConvertor implements Trigger.TriggerContext { |
| 143 | + |
| 144 | + private final AsyncTrigger.TriggerContext asyncTriggerContext; |
| 145 | + |
| 146 | + private AsyncTriggerContextConvertor(AsyncTrigger.TriggerContext asyncTriggerContext) { |
| 147 | + this.asyncTriggerContext = asyncTriggerContext; |
| 148 | + } |
| 149 | + |
| 150 | + @Override |
| 151 | + public long getCurrentProcessingTime() { |
| 152 | + return asyncTriggerContext.getCurrentProcessingTime(); |
| 153 | + } |
| 154 | + |
| 155 | + @Override |
| 156 | + public MetricGroup getMetricGroup() { |
| 157 | + return asyncTriggerContext.getMetricGroup(); |
| 158 | + } |
| 159 | + |
| 160 | + @Override |
| 161 | + public long getCurrentWatermark() { |
| 162 | + return asyncTriggerContext.getCurrentWatermark(); |
| 163 | + } |
| 164 | + |
| 165 | + @Override |
| 166 | + public void registerProcessingTimeTimer(long time) { |
| 167 | + asyncTriggerContext.registerProcessingTimeTimer(time); |
| 168 | + } |
| 169 | + |
| 170 | + @Override |
| 171 | + public void registerEventTimeTimer(long time) { |
| 172 | + asyncTriggerContext.registerEventTimeTimer(time); |
| 173 | + } |
| 174 | + |
| 175 | + @Override |
| 176 | + public void deleteProcessingTimeTimer(long time) { |
| 177 | + asyncTriggerContext.deleteProcessingTimeTimer(time); |
| 178 | + } |
| 179 | + |
| 180 | + @Override |
| 181 | + public void deleteEventTimeTimer(long time) { |
| 182 | + asyncTriggerContext.deleteEventTimeTimer(time); |
| 183 | + } |
| 184 | + |
| 185 | + @Override |
| 186 | + public <S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor) { |
| 187 | + throw new UnsupportedOperationException( |
| 188 | + "Trigger is for state V1 APIs, window operator with async state enabled only accept state V2 APIs."); |
| 189 | + } |
| 190 | + |
| 191 | + public static Trigger.TriggerContext of( |
| 192 | + AsyncTrigger.TriggerContext asyncTriggerContext) { |
| 193 | + return new AsyncTriggerContextConvertor(asyncTriggerContext); |
| 194 | + } |
| 195 | + } |
| 196 | + |
| 197 | + @VisibleForTesting |
| 198 | + public Trigger<T, W> getUserDefinedTrigger() { |
| 199 | + return userDefinedTrigger; |
| 200 | + } |
| 201 | + } |
| 202 | +} |
0 commit comments