|
18 | 18 | */ |
19 | 19 | package org.apache.samza.operators.impl; |
20 | 20 |
|
| 21 | +import org.apache.samza.config.Config; |
| 22 | +import org.apache.samza.config.MetricsConfig; |
| 23 | +import org.apache.samza.metrics.Counter; |
| 24 | +import org.apache.samza.metrics.MetricsRegistry; |
| 25 | +import org.apache.samza.metrics.Timer; |
| 26 | +import org.apache.samza.operators.spec.OperatorSpec; |
21 | 27 | import org.apache.samza.task.MessageCollector; |
| 28 | +import org.apache.samza.task.TaskContext; |
22 | 29 | import org.apache.samza.task.TaskCoordinator; |
| 30 | +import org.apache.samza.util.HighResolutionClock; |
23 | 31 |
|
| 32 | +import java.util.Collection; |
| 33 | +import java.util.Collections; |
24 | 34 | import java.util.HashSet; |
25 | 35 | import java.util.Set; |
26 | 36 |
|
|
29 | 39 | * Abstract base class for all stream operator implementations. |
30 | 40 | */ |
31 | 41 | public abstract class OperatorImpl<M, RM> { |
| 42 | + private static final String METRICS_GROUP = OperatorImpl.class.getName(); |
32 | 43 |
|
33 | | - private final Set<OperatorImpl<RM, ?>> nextOperators = new HashSet<>(); |
| 44 | + private boolean initialized; |
| 45 | + private Set<OperatorImpl<RM, ?>> registeredOperators; |
| 46 | + private HighResolutionClock highResClock; |
| 47 | + private Counter numMessage; |
| 48 | + private Timer handleMessageNs; |
| 49 | + private Timer handleTimerNs; |
34 | 50 |
|
35 | 51 | /** |
36 | | - * Register the next operator in the chain that this operator should propagate its output to. |
37 | | - * @param nextOperator the next operator in the chain. |
| 52 | + * Initialize this {@link OperatorImpl} and its user-defined functions. |
| 53 | + * |
| 54 | + * @param config the {@link Config} for the task |
| 55 | + * @param context the {@link TaskContext} for the task |
| 56 | + */ |
| 57 | + public final void init(Config config, TaskContext context) { |
| 58 | + String opName = getOperatorSpec().getOpName(); |
| 59 | + |
| 60 | + if (initialized) { |
| 61 | + throw new IllegalStateException(String.format("Attempted to initialize Operator %s more than once.", opName)); |
| 62 | + } |
| 63 | + |
| 64 | + this.highResClock = createHighResClock(config); |
| 65 | + registeredOperators = new HashSet<>(); |
| 66 | + MetricsRegistry metricsRegistry = context.getMetricsRegistry(); |
| 67 | + this.numMessage = metricsRegistry.newCounter(METRICS_GROUP, opName + "-messages"); |
| 68 | + this.handleMessageNs = metricsRegistry.newTimer(METRICS_GROUP, opName + "-handle-message-ns"); |
| 69 | + this.handleTimerNs = metricsRegistry.newTimer(METRICS_GROUP, opName + "-handle-timer-ns"); |
| 70 | + |
| 71 | + handleInit(config, context); |
| 72 | + |
| 73 | + initialized = true; |
| 74 | + } |
| 75 | + |
| 76 | + /** |
| 77 | + * Initialize this {@link OperatorImpl} and its user-defined functions. |
| 78 | + * |
| 79 | + * @param config the {@link Config} for the task |
| 80 | + * @param context the {@link TaskContext} for the task |
| 81 | + */ |
| 82 | + protected abstract void handleInit(Config config, TaskContext context); |
| 83 | + |
| 84 | + /** |
| 85 | + * Register an operator that this operator should propagate its results to. |
| 86 | + * |
| 87 | + * @param nextOperator the next operator to propagate results to |
38 | 88 | */ |
39 | 89 | void registerNextOperator(OperatorImpl<RM, ?> nextOperator) { |
40 | | - nextOperators.add(nextOperator); |
| 90 | + if (!initialized) { |
| 91 | + throw new IllegalStateException( |
| 92 | + String.format("Attempted to register next operator before initializing operator %s.", |
| 93 | + getOperatorSpec().getOpName())); |
| 94 | + } |
| 95 | + this.registeredOperators.add(nextOperator); |
41 | 96 | } |
42 | 97 |
|
43 | 98 | /** |
44 | | - * Perform the transformation required for this operator and call the downstream operators. |
| 99 | + * Handle the incoming {@code message} for this {@link OperatorImpl} and propagate results to registered operators. |
| 100 | + * <p> |
| 101 | + * Delegates to {@link #handleMessage(Object, MessageCollector, TaskCoordinator)} for handling the message. |
45 | 102 | * |
46 | | - * Must call {@link #propagateResult} to propagate the output to registered downstream operators correctly. |
| 103 | + * @param message the input message |
| 104 | + * @param collector the {@link MessageCollector} for this message |
| 105 | + * @param coordinator the {@link TaskCoordinator} for this message |
| 106 | + */ |
| 107 | + public final void onMessage(M message, MessageCollector collector, TaskCoordinator coordinator) { |
| 108 | + this.numMessage.inc(); |
| 109 | + long startNs = this.highResClock.nanoTime(); |
| 110 | + Collection<RM> results = handleMessage(message, collector, coordinator); |
| 111 | + long endNs = this.highResClock.nanoTime(); |
| 112 | + this.handleMessageNs.update(endNs - startNs); |
| 113 | + |
| 114 | + results.forEach(rm -> |
| 115 | + this.registeredOperators.forEach(op -> |
| 116 | + op.onMessage(rm, collector, coordinator))); |
| 117 | + } |
| 118 | + |
| 119 | + /** |
| 120 | + * Handle the incoming {@code message} and return the results to be propagated to registered operators. |
47 | 121 | * |
48 | 122 | * @param message the input message |
49 | 123 | * @param collector the {@link MessageCollector} in the context |
50 | 124 | * @param coordinator the {@link TaskCoordinator} in the context |
| 125 | + * @return results of the transformation |
51 | 126 | */ |
52 | | - public abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator); |
| 127 | + protected abstract Collection<RM> handleMessage(M message, MessageCollector collector, |
| 128 | + TaskCoordinator coordinator); |
53 | 129 |
|
54 | 130 | /** |
55 | | - * Invoked at every tick. This method delegates to {@link #onTimer(MessageCollector, TaskCoordinator)} |
| 131 | + * Handle timer ticks for this {@link OperatorImpl} and propagate the results and timer tick to registered operators. |
| 132 | + * <p> |
| 133 | + * Delegates to {@link #handleTimer(MessageCollector, TaskCoordinator)} for handling the timer tick. |
56 | 134 | * |
57 | 135 | * @param collector the {@link MessageCollector} in the context |
58 | 136 | * @param coordinator the {@link TaskCoordinator} in the context |
59 | 137 | */ |
60 | | - public final void onTick(MessageCollector collector, TaskCoordinator coordinator) { |
61 | | - onTimer(collector, coordinator); |
62 | | - nextOperators.forEach(sub -> sub.onTick(collector, coordinator)); |
| 138 | + public final void onTimer(MessageCollector collector, TaskCoordinator coordinator) { |
| 139 | + long startNs = this.highResClock.nanoTime(); |
| 140 | + Collection<RM> results = handleTimer(collector, coordinator); |
| 141 | + long endNs = this.highResClock.nanoTime(); |
| 142 | + this.handleTimerNs.update(endNs - startNs); |
| 143 | + |
| 144 | + results.forEach(rm -> |
| 145 | + this.registeredOperators.forEach(op -> |
| 146 | + op.onMessage(rm, collector, coordinator))); |
| 147 | + this.registeredOperators.forEach(op -> |
| 148 | + op.onTimer(collector, coordinator)); |
63 | 149 | } |
64 | 150 |
|
65 | 151 | /** |
66 | | - * Invoked at every tick. Implementations must call {@link #propagateResult} to propagate any generated output |
67 | | - * to registered downstream operators. |
| 152 | + * Handle the the timer tick for this operator and return the results to be propagated to registered operators. |
| 153 | + * <p> |
| 154 | + * Defaults to a no-op implementation. |
68 | 155 | * |
69 | 156 | * @param collector the {@link MessageCollector} in the context |
70 | 157 | * @param coordinator the {@link TaskCoordinator} in the context |
| 158 | + * @return results of the timed operation |
71 | 159 | */ |
72 | | - public void onTimer(MessageCollector collector, TaskCoordinator coordinator) { |
| 160 | + protected Collection<RM> handleTimer(MessageCollector collector, TaskCoordinator coordinator) { |
| 161 | + return Collections.emptyList(); |
73 | 162 | } |
74 | 163 |
|
75 | 164 | /** |
76 | | - * Helper method to propagate the output of this operator to all registered downstream operators. |
77 | | - * |
78 | | - * This method <b>must</b> be called from {@link #onNext} and {@link #onTimer} |
79 | | - * to propagate the operator output correctly. |
| 165 | + * Get the {@link OperatorSpec} for this {@link OperatorImpl}. |
80 | 166 | * |
81 | | - * @param outputMessage output message |
82 | | - * @param collector the {@link MessageCollector} in the context |
83 | | - * @param coordinator the {@link TaskCoordinator} in the context |
| 167 | + * @return the {@link OperatorSpec} for this {@link OperatorImpl} |
84 | 168 | */ |
85 | | - void propagateResult(RM outputMessage, MessageCollector collector, TaskCoordinator coordinator) { |
86 | | - nextOperators.forEach(sub -> sub.onNext(outputMessage, collector, coordinator)); |
| 169 | + protected abstract OperatorSpec<RM> getOperatorSpec(); |
| 170 | + |
| 171 | + private HighResolutionClock createHighResClock(Config config) { |
| 172 | + if (new MetricsConfig(config).getMetricsTimerEnabled()) { |
| 173 | + return System::nanoTime; |
| 174 | + } else { |
| 175 | + return () -> 0; |
| 176 | + } |
87 | 177 | } |
88 | 178 | } |
0 commit comments