|
| 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 com.dtstack.flink.sql.environment; |
| 20 | + |
| 21 | +import com.dtstack.flink.sql.constrant.ConfigConstrant; |
| 22 | +import com.dtstack.flink.sql.util.FlinkUtil; |
| 23 | +import com.dtstack.flink.sql.util.PropertiesUtils; |
| 24 | +import org.apache.commons.lang3.StringUtils; |
| 25 | +import org.apache.flink.api.common.ExecutionConfig; |
| 26 | +import org.apache.flink.api.common.restartstrategy.RestartStrategies; |
| 27 | +import org.apache.flink.api.common.time.Time; |
| 28 | +import org.apache.flink.configuration.Configuration; |
| 29 | +import org.apache.flink.streaming.api.TimeCharacteristic; |
| 30 | +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 31 | +import org.apache.flink.table.api.TableEnvironment; |
| 32 | + |
| 33 | +import java.io.IOException; |
| 34 | +import java.lang.reflect.InvocationTargetException; |
| 35 | +import java.lang.reflect.Method; |
| 36 | +import java.util.Arrays; |
| 37 | +import java.util.Map; |
| 38 | +import java.util.Optional; |
| 39 | +import java.util.Properties; |
| 40 | +import java.util.concurrent.TimeUnit; |
| 41 | + |
| 42 | +/** |
| 43 | + * |
| 44 | + * 流执行环境相关配置 |
| 45 | + * Date: 2019/11/22 |
| 46 | + * Company: www.dtstack.com |
| 47 | + * @author maqi |
| 48 | + */ |
| 49 | +public final class StreamEnvConfigManager { |
| 50 | + private StreamEnvConfigManager() { |
| 51 | + throw new AssertionError("Singleton class."); |
| 52 | + } |
| 53 | + |
| 54 | + /** |
| 55 | + * 配置StreamExecutionEnvironment运行时参数 |
| 56 | + * @param streamEnv |
| 57 | + * @param confProperties |
| 58 | + */ |
| 59 | + public static void streamExecutionEnvironmentConfig(StreamExecutionEnvironment streamEnv, Properties confProperties) |
| 60 | + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException, IOException { |
| 61 | + |
| 62 | + confProperties = PropertiesUtils.propertiesTrim(confProperties); |
| 63 | + streamEnv.getConfig().disableClosureCleaner(); |
| 64 | + |
| 65 | + Configuration globalJobParameters = new Configuration(); |
| 66 | + //Configuration unsupported set properties key-value |
| 67 | + Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); |
| 68 | + method.setAccessible(true); |
| 69 | + for (Map.Entry<Object, Object> prop : confProperties.entrySet()) { |
| 70 | + method.invoke(globalJobParameters, prop.getKey(), prop.getValue()); |
| 71 | + } |
| 72 | + |
| 73 | + ExecutionConfig exeConfig = streamEnv.getConfig(); |
| 74 | + if (exeConfig.getGlobalJobParameters() == null) { |
| 75 | + exeConfig.setGlobalJobParameters(globalJobParameters); |
| 76 | + } else if (exeConfig.getGlobalJobParameters() instanceof Configuration) { |
| 77 | + ((Configuration) exeConfig.getGlobalJobParameters()).addAll(globalJobParameters); |
| 78 | + } |
| 79 | + |
| 80 | + Optional<Integer> envParallelism = getEnvParallelism(confProperties); |
| 81 | + if (envParallelism.isPresent()) { |
| 82 | + streamEnv.setParallelism(envParallelism.get()); |
| 83 | + } |
| 84 | + |
| 85 | + Optional<Integer> maxParallelism = getMaxEnvParallelism(confProperties); |
| 86 | + if (maxParallelism.isPresent()) { |
| 87 | + streamEnv.setMaxParallelism(maxParallelism.get()); |
| 88 | + } |
| 89 | + |
| 90 | + Optional<Long> bufferTimeoutMillis = getBufferTimeoutMillis(confProperties); |
| 91 | + if (bufferTimeoutMillis.isPresent()) { |
| 92 | + streamEnv.setBufferTimeout(bufferTimeoutMillis.get()); |
| 93 | + } |
| 94 | + |
| 95 | + Optional<TimeCharacteristic> streamTimeCharacteristic = getStreamTimeCharacteristic(confProperties); |
| 96 | + if (streamTimeCharacteristic.isPresent()) { |
| 97 | + streamEnv.setStreamTimeCharacteristic(streamTimeCharacteristic.get()); |
| 98 | + } |
| 99 | + |
| 100 | + streamEnv.setRestartStrategy(RestartStrategies.failureRateRestart( |
| 101 | + ConfigConstrant.failureRate, |
| 102 | + Time.of(ConfigConstrant.failureInterval, TimeUnit.MINUTES), |
| 103 | + Time.of(ConfigConstrant.delayInterval, TimeUnit.SECONDS) |
| 104 | + )); |
| 105 | + |
| 106 | + FlinkUtil.openCheckpoint(streamEnv, confProperties); |
| 107 | + } |
| 108 | + |
| 109 | + public static void streamTableEnvironmentStateTTLConfig(TableEnvironment tableEnv, Properties confProperties) { |
| 110 | + confProperties = PropertiesUtils.propertiesTrim(confProperties); |
| 111 | + FlinkUtil.setTableEnvTTL(confProperties, tableEnv); |
| 112 | + } |
| 113 | + |
| 114 | + public static Optional<Integer> getEnvParallelism(Properties properties) { |
| 115 | + String parallelismStr = properties.getProperty(ConfigConstrant.SQL_ENV_PARALLELISM); |
| 116 | + return StringUtils.isNotBlank(parallelismStr) ? Optional.of(Integer.valueOf(parallelismStr)) : Optional.empty(); |
| 117 | + } |
| 118 | + |
| 119 | + public static Optional<Integer> getMaxEnvParallelism(Properties properties) { |
| 120 | + String parallelismStr = properties.getProperty(ConfigConstrant.SQL_MAX_ENV_PARALLELISM); |
| 121 | + return StringUtils.isNotBlank(parallelismStr) ? Optional.of(Integer.valueOf(parallelismStr)) : Optional.empty(); |
| 122 | + } |
| 123 | + |
| 124 | + public static Optional<Long> getBufferTimeoutMillis(Properties properties) { |
| 125 | + String mills = properties.getProperty(ConfigConstrant.SQL_BUFFER_TIMEOUT_MILLIS); |
| 126 | + return StringUtils.isNotBlank(mills) ? Optional.of(Long.valueOf(mills)) : Optional.empty(); |
| 127 | + } |
| 128 | + |
| 129 | + /** |
| 130 | + * #ProcessingTime(默认), IngestionTime, EventTime |
| 131 | + * @param properties |
| 132 | + */ |
| 133 | + public static Optional<TimeCharacteristic> getStreamTimeCharacteristic(Properties properties) { |
| 134 | + if (!properties.containsKey(ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY)) { |
| 135 | + return Optional.empty(); |
| 136 | + } |
| 137 | + String characteristicStr = properties.getProperty(ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY); |
| 138 | + Optional<TimeCharacteristic> characteristic = Arrays.stream(TimeCharacteristic.values()) |
| 139 | + .filter(tc -> !characteristicStr.equalsIgnoreCase(tc.toString())).findAny(); |
| 140 | + |
| 141 | + if (!characteristic.isPresent()) { |
| 142 | + throw new RuntimeException("illegal property :" + ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY); |
| 143 | + } |
| 144 | + return characteristic; |
| 145 | + } |
| 146 | +} |
0 commit comments