Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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
2 changes: 1 addition & 1 deletion examples/autoscaling/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -16,5 +16,5 @@
# limitations under the License.
################################################################################

FROM flink:1.18
FROM flink:1.20.1-java17
COPY ./target/autoscaling*.jar /opt/flink/usrlib/autoscaling.jar
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;

/** Autoscaling Example. */
public class AutoscalingExample {
Expand All @@ -43,13 +43,7 @@ public Long map(Long i) throws Exception {
return end;
}
});
stream.addSink(
new SinkFunction<Long>() {
@Override
public void invoke(Long value, Context context) throws Exception {
// Do nothing
}
});
stream.sinkTo(new DiscardingSink<>());
env.execute("Autoscaling Example");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import org.apache.flink.connector.datagen.source.GeneratorFunction;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.util.Collector;

import org.slf4j.Logger;
Expand Down Expand Up @@ -128,7 +128,7 @@ public static void main(String[] args) throws Exception {
.broadcast();
}

stream.addSink(new DiscardingSink<>());
stream.sinkTo(new DiscardingSink<>());
}

env.execute(
Expand Down Expand Up @@ -164,15 +164,16 @@ public void flatMap(Long record, Collector<Long> out) throws Exception {

double amplitude = getAmplitude(currentEpoch);

double loadPerSubTask = maxLoad / getRuntimeContext().getNumberOfParallelSubtasks();
double loadPerSubTask =
maxLoad / getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks();

long busyTimeMs = (long) (loadPerSubTask * samplingIntervalMs * amplitude);
long remainingTimeMs =
(timeMillis / samplingIntervalMs + 1) * samplingIntervalMs - timeMillis;
long sleepTime = Math.min(busyTimeMs, remainingTimeMs);
LOG.info(
"{}> epoch: {} busyTime: {} remainingTime: {} sleepTime: {} amplitude: {}",
getRuntimeContext().getIndexOfThisSubtask(),
getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(),
currentEpoch,
busyTimeMs,
remainingTimeMs,
Expand Down
Loading