From 96e333716adb68edc0afde6bf4123ae6b4b76ae1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 16 Nov 2018 21:42:20 +0800 Subject: [PATCH 01/42] add per job submit mode --- .../sql/launcher/ClusterClientFactory.java | 16 ++ .../flink/sql/launcher/LauncherMain.java | 2 +- .../perjob/FLinkPerJobResourceUtil.java | 80 ++++++++++ .../perjob/PerJobClusterClientBuilder.java | 145 ++++++++++++++++++ 4 files changed, 242 insertions(+), 1 deletion(-) create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 108ec0989..61702ea76 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,6 +18,8 @@ package com.dtstack.flink.sql.launcher; +import com.dtstack.flink.sql.launcher.perjob.FLinkPerJobResourceUtil; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.ClusterSpecification; @@ -45,12 +47,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -71,7 +75,10 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) return createStandaloneClient(launcherOptions); } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarnPer.name())){ + return createPerJobModeYarnClient(launcherOptions); } + throw new IllegalArgumentException("Unsupported cluster client type: "); } @@ -172,6 +179,15 @@ public boolean accept(File dir, String name) { throw new UnsupportedOperationException("Haven't been developed yet!"); } + public static ClusterClient createPerJobModeYarnClient(LauncherOptions launcherOptions) throws IOException { + + Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); + + //TODO + return null; + } + /** * 处理yarn HA的配置项 */ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 55f085c98..4553bd2ab 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -55,7 +55,7 @@ public static void main(String[] args) throws Exception { if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - } else { + } else{ ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java new file mode 100644 index 000000000..3becc41d8 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java @@ -0,0 +1,80 @@ +/** + * 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 com.dtstack.flink.sql.launcher.perjob; + + +import com.dtstack.flink.sql.util.MathUtil; +import org.apache.flink.client.deployment.ClusterSpecification; + +import java.util.Properties; + +/** + * company: www.dtstack.com + * author xuchao + * create: 2018/7/5 + */ +public class FLinkPerJobResourceUtil { + + public final static int MIN_JM_MEMORY = 1024; // the minimum memory should be higher than the min heap cutoff + public final static int MIN_TM_MEMORY = 1024; + + public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; + public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; + public final static String CONTAINER = "container"; + public final static String SLOTS = "slots"; + + public static ClusterSpecification createClusterSpecification(Properties confProperties) { + int jobmanagerMemoryMb = 1024; + int taskmanagerMemoryMb = 1024; + int numberTaskManagers = 1; + int slotsPerTaskManager = 1; + + if (confProperties != null) { + if (confProperties.containsKey(JOBMANAGER_MEMORY_MB)){ + jobmanagerMemoryMb = MathUtil.getIntegerVal(confProperties.get(JOBMANAGER_MEMORY_MB)); + if (jobmanagerMemoryMb < MIN_JM_MEMORY) { + jobmanagerMemoryMb = MIN_JM_MEMORY; + } + } + + if (confProperties.containsKey(TASKMANAGER_MEMORY_MB)){ + taskmanagerMemoryMb = MathUtil.getIntegerVal(confProperties.get(TASKMANAGER_MEMORY_MB)); + if (taskmanagerMemoryMb < MIN_TM_MEMORY) { + taskmanagerMemoryMb = MIN_TM_MEMORY; + } + } + + if (confProperties.containsKey(CONTAINER)){ + numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(CONTAINER)); + } + + if (confProperties.containsKey(SLOTS)){ + slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS)); + } + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobmanagerMemoryMb) + .setTaskManagerMemoryMB(taskmanagerMemoryMb) + .setNumberTaskManagers(numberTaskManagers) + .setSlotsPerTaskManager(slotsPerTaskManager) + .createClusterSpecification(); + } + +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java new file mode 100644 index 000000000..6531ff5b0 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -0,0 +1,145 @@ +/** + * 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 com.dtstack.flink.sql.launcher.perjob; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Reason: + * Date: 2018/11/16 + * Company: www.dtstack.com + * @author xuchao + */ + +public class PerJobClusterClientBuilder { + + public static final String DEFAULT_GATEWAY_CLASS = "org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"; + + public static final String PROMGATEWAY_CLASS_KEY = "metrics.reporter.promgateway.class"; + + public static final String PROMGATEWAY_HOST_KEY = "metrics.reporter.promgateway.host"; + + public static final String PROMGATEWAY_PORT_KEY = "metrics.reporter.promgateway.port"; + + public static final String PROMGATEWAY_JOBNAME_KEY = "metrics.reporter.promgateway.jobName"; + + public static final String PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY = "metrics.reporter.promgateway.randomJobNameSuffix"; + + public static final String PROMGATEWAY_DELETEONSHUTDOWN_KEY = "metrics.reporter.promgateway.deleteOnShutdown"; + + private YarnClient yarnClient; + + private YarnConfiguration yarnConf; + + public void init(String yarnConfDir){ + if(Strings.isNullOrEmpty(yarnConfDir)) { + throw new RuntimeException("parameters of yarn is required"); + } + + } + + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties properties, String flinkJarPath, + String jobmanagerArchiveFsDir, String queue) throws MalformedURLException { + Configuration newConf = new Configuration(); + + if(properties.containsKey(HighAvailabilityOptions.HA_CLUSTER_ID)){ + newConf.setString(HighAvailabilityOptions.HA_CLUSTER_ID, properties.getProperty(HighAvailabilityOptions.HA_CLUSTER_ID.key())); + } + + perJobMetricConfigConfig(newConf, properties); + + AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); + + if (StringUtils.isNotBlank(flinkJarPath)) { + + if (!new File(flinkJarPath).exists()) { + throw new RuntimeException("The Flink jar path is not exist"); + } + + } + + if(StringUtils.isNotBlank(jobmanagerArchiveFsDir)){ + newConf.setString(JobManagerOptions.ARCHIVE_DIR, jobmanagerArchiveFsDir); + } + + List classpaths = new ArrayList<>(); + if (flinkJarPath != null) { + File[] jars = new File(flinkJarPath).listFiles(); + + for (File file : jars){ + if (file.toURI().toURL().toString().contains("flink-dist")){ + clusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); + } else { + classpaths.add(file.toURI().toURL()); + } + } + + } else { + throw new RuntimeException("The Flink jar path is null"); + } + + clusterDescriptor.setProvidedUserJarFiles(classpaths); + + if(!Strings.isNullOrEmpty(queue)){ + clusterDescriptor.setQueue(queue); + } + return clusterDescriptor; + } + + private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ + if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ + return; + } + + configuration.setString(PROMGATEWAY_CLASS_KEY, properties.getProperty(PROMGATEWAY_CLASS_KEY)); + configuration.setString(PROMGATEWAY_HOST_KEY, properties.getProperty(PROMGATEWAY_HOST_KEY)); + configuration.setString(PROMGATEWAY_PORT_KEY, properties.getProperty(PROMGATEWAY_PORT_KEY)); + configuration.setString(PROMGATEWAY_JOBNAME_KEY, properties.getProperty(PROMGATEWAY_JOBNAME_KEY)); + configuration.setString(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY, properties.getProperty(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY)); + configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); + } + + public AbstractYarnClusterDescriptor getClusterDescriptor( + Configuration configuration, + YarnConfiguration yarnConfiguration, + String configurationDirectory) { + return new YarnClusterDescriptor( + configuration, + yarnConfiguration, + configurationDirectory, + yarnClient, + false); + } +} From 1ef29b0dc3e80cfa5a737af7e5d8c6ae2c9244c6 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 14:33:23 +0800 Subject: [PATCH 02/42] add per job submit mode --- .../sql/launcher/ClusterClientFactory.java | 167 ++++-------------- .../flink/sql/launcher/LauncherMain.java | 33 +++- .../sql/launcher/LauncherOptionParser.java | 11 +- .../flink/sql/launcher/LauncherOptions.java | 19 ++ .../flink/sql/launcher/YarnConfLoader.java | 85 +++++++++ .../perjob/PerJobClusterClientBuilder.java | 27 ++- .../sql/launcher/perjob/PerJobSubmitter.java | 66 +++++++ 7 files changed, 253 insertions(+), 155 deletions(-) create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 61702ea76..9dfe2e07e 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,15 +18,11 @@ package com.dtstack.flink.sql.launcher; -import com.dtstack.flink.sql.launcher.perjob.FLinkPerJobResourceUtil; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.ClusterMode; import org.apache.commons.lang.StringUtils; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -34,9 +30,6 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -45,21 +38,12 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import java.io.File; -import java.io.FilenameFilter; -import java.io.IOException; + import java.net.InetSocketAddress; import java.util.EnumSet; import java.util.HashSet; -import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Properties; import java.util.Set; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; - -import com.dtstack.flink.sql.ClusterMode; /** * The Factory of ClusterClient @@ -75,8 +59,6 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) return createStandaloneClient(launcherOptions); } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarnPer.name())){ - return createPerJobModeYarnClient(launcherOptions); } throw new IllegalArgumentException("Unsupported cluster client type: "); @@ -101,138 +83,61 @@ public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); - YarnConfiguration yarnConf = new YarnConfiguration(); if(StringUtils.isNotBlank(yarnConfDir)) { - try { + try { config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); FileSystem.initialize(config); - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } + YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConf); + yarnClient.start(); + ApplicationId applicationId = null; + + Set set = new HashSet<>(); + set.add("Apache Flink"); + EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); + enumSet.add(YarnApplicationState.RUNNING); + List reportList = yarnClient.getApplications(set, enumSet); + + int maxMemory = -1; + int maxCores = -1; + for(ApplicationReport report : reportList) { + if(!report.getName().startsWith("Flink session")){ + continue; } - YarnClient yarnClient = YarnClient.createYarnClient(); - haYarnConf(yarnConf); - yarnClient.init(yarnConf); - yarnClient.start(); - ApplicationId applicationId = null; - - Set set = new HashSet<>(); - set.add("Apache Flink"); - EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); - enumSet.add(YarnApplicationState.RUNNING); - List reportList = yarnClient.getApplications(set, enumSet); - - int maxMemory = -1; - int maxCores = -1; - for(ApplicationReport report : reportList) { - if(!report.getName().startsWith("Flink session")){ - continue; - } - - if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { - continue; - } - - int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); - int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); - if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { - maxMemory = thisMemory; - maxCores = thisCores; - applicationId = report.getApplicationId(); - } - + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; } - if(StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); + int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); + if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { + maxMemory = thisMemory; + maxCores = thisCores; + applicationId = report.getApplicationId(); } + } - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); - ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); - clusterClient.setDetached(true); - return clusterClient; + if(StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); } + + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); + ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + clusterClient.setDetached(true); + return clusterClient; } catch(Exception e) { throw new RuntimeException(e); } } - - throw new UnsupportedOperationException("Haven't been developed yet!"); } - public static ClusterClient createPerJobModeYarnClient(LauncherOptions launcherOptions) throws IOException { - - Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); - ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); - //TODO - return null; - } - - /** - * 处理yarn HA的配置项 - */ - private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop.conf.Configuration yarnConf) { - Iterator> iterator = yarnConf.iterator(); - while(iterator.hasNext()) { - Map.Entry entry = iterator.next(); - String key = entry.getKey(); - String value = entry.getValue(); - if(key.startsWith("yarn.resourcemanager.hostname.")) { - String rm = key.substring("yarn.resourcemanager.hostname.".length()); - String addressKey = "yarn.resourcemanager.address." + rm; - if(yarnConf.get(addressKey) == null) { - yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); - } - } - } - return yarnConf; - } - - private static org.apache.hadoop.conf.Configuration getYarnConf(String yarnConfDir) { - org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); - try { - - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } - } - } - } catch(Exception e) { - throw new RuntimeException(e); - } - return yarnConf; - } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 4553bd2ab..8e5832294 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -22,11 +22,16 @@ import avro.shaded.com.google.common.collect.Lists; import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import java.io.File; import java.util.List; import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; @@ -55,17 +60,29 @@ public static void main(String[] args) throws Exception { if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - } else{ + return; + } + + String pluginRoot = launcherOptions.getLocalSqlPluginPath(); + File jarFile = new File(getLocalCoreJarPath(pluginRoot)); + String[] remoteArgs = argList.toArray(new String[argList.size()]); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + + if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ + program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); + } + + if(mode.equals(ClusterMode.yarnPer.name())){ + String flinkConfDir = launcherOptions.getFlinkconf(); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, 1); + PerJobSubmitter.submit(launcherOptions, jobGraph); + } else { ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - String pluginRoot = launcherOptions.getLocalSqlPluginPath(); - File jarFile = new File(getLocalCoreJarPath(pluginRoot)); - String[] remoteArgs = argList.toArray(new String[argList.size()]); - PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); - if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); - } clusterClient.run(program, 1); clusterClient.shutdown(); } + + System.out.println("---submit end----"); } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 5052d22e8..75c5c4f0f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -65,6 +65,8 @@ public class LauncherOptionParser { public static final String OPTION_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; + public static final String OPTION_FLINK_JAR_PATH = "flinkJarPath"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); @@ -84,6 +86,7 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); + options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); try { CommandLine cl = parser.parse(options, args); @@ -137,6 +140,11 @@ public LauncherOptionParser(String[] args) { properties.setAllowNonRestoredState(allow_non); } + String flinkJarPath = cl.getOptionValue(OPTION_FLINK_JAR_PATH); + if(StringUtils.isNotBlank(flinkJarPath)){ + properties.setFlinkJarPath(flinkJarPath); + } + } catch (Exception e) { throw new RuntimeException(e); } @@ -152,7 +160,8 @@ public List getProgramExeArgList() throws Exception { for(Map.Entry one : mapConf.entrySet()){ String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) - || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ + || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key) + || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key)){ continue; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index f183957f7..5cc762ebd 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -48,6 +48,11 @@ public class LauncherOptions { private String allowNonRestoredState = "false"; + //just use for per_job mode + private String flinkJarPath; + + private String queue; + public String getMode() { return mode; } @@ -136,5 +141,19 @@ public void setAllowNonRestoredState(String allowNonRestoredState) { this.allowNonRestoredState = allowNonRestoredState; } + public String getFlinkJarPath() { + return flinkJarPath; + } + + public void setFlinkJarPath(String flinkJarPath) { + this.flinkJarPath = flinkJarPath; + } + + public String getQueue() { + return queue; + } + public void setQueue(String queue) { + this.queue = queue; + } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java new file mode 100644 index 000000000..717da24fa --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java @@ -0,0 +1,85 @@ +/* + * 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 com.dtstack.flink.sql.launcher; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.util.Iterator; +import java.util.Map; + +/** + * load yarn conf from specify dir + * Date: 2018/11/17 + * Company: www.dtstack.com + * @author xuchao + */ + +public class YarnConfLoader { + + public static YarnConfiguration getYarnConf(String yarnConfDir) { + YarnConfiguration yarnConf = new YarnConfiguration(); + try { + + File dir = new File(yarnConfDir); + if(dir.exists() && dir.isDirectory()) { + + File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { + if(name.endsWith(".xml")){ + return true; + } + return false; + }); + + if(xmlFileList != null) { + for(File xmlFile : xmlFileList) { + yarnConf.addResource(xmlFile.toURI().toURL()); + } + } + } + + } catch(Exception e) { + throw new RuntimeException(e); + } + + haYarnConf(yarnConf); + return yarnConf; + } + + /** + * deal yarn HA conf + */ + private static Configuration haYarnConf(Configuration yarnConf) { + Iterator> iterator = yarnConf.iterator(); + while(iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String key = entry.getKey(); + String value = entry.getValue(); + if(key.startsWith("yarn.resourcemanager.hostname.")) { + String rm = key.substring("yarn.resourcemanager.hostname.".length()); + String addressKey = "yarn.resourcemanager.address." + rm; + if(yarnConf.get(addressKey) == null) { + yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); + } + } + } + return yarnConf; + } +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 6531ff5b0..7d395f0ac 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -18,10 +18,9 @@ package com.dtstack.flink.sql.launcher.perjob; +import com.dtstack.flink.sql.launcher.YarnConfLoader; import org.apache.commons.lang3.StringUtils; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -68,18 +67,19 @@ public void init(String yarnConfDir){ throw new RuntimeException("parameters of yarn is required"); } + yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConf); + yarnClient.start(); + + System.out.println("----init yarn success ----"); } - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties properties, String flinkJarPath, - String jobmanagerArchiveFsDir, String queue) throws MalformedURLException { + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); + newConf.addAllToProperties(confProp); - if(properties.containsKey(HighAvailabilityOptions.HA_CLUSTER_ID)){ - newConf.setString(HighAvailabilityOptions.HA_CLUSTER_ID, properties.getProperty(HighAvailabilityOptions.HA_CLUSTER_ID.key())); - } - - perJobMetricConfigConfig(newConf, properties); - + //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); if (StringUtils.isNotBlank(flinkJarPath)) { @@ -90,10 +90,6 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties pr } - if(StringUtils.isNotBlank(jobmanagerArchiveFsDir)){ - newConf.setString(JobManagerOptions.ARCHIVE_DIR, jobmanagerArchiveFsDir); - } - List classpaths = new ArrayList<>(); if (flinkJarPath != null) { File[] jars = new File(flinkJarPath).listFiles(); @@ -118,6 +114,7 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties pr return clusterDescriptor; } + //FIXME need? private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ return; @@ -131,7 +128,7 @@ private void perJobMetricConfigConfig(Configuration configuration, Properties pr configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); } - public AbstractYarnClusterDescriptor getClusterDescriptor( + private AbstractYarnClusterDescriptor getClusterDescriptor( Configuration configuration, YarnConfiguration yarnConfiguration, String configurationDirectory) { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java new file mode 100644 index 000000000..3cdf3f8d4 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -0,0 +1,66 @@ +/* + * 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 com.dtstack.flink.sql.launcher.perjob; + +import com.dtstack.flink.sql.launcher.LauncherOptions; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * per job mode submitter + * Date: 2018/11/17 + * Company: www.dtstack.com + * @author xuchao + */ + +public class PerJobSubmitter { + + private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); + + public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { + + Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); + + PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); + perJobClusterClientBuilder.init(launcherOptions.getYarnconf()); + + String flinkJarPath = launcherOptions.getFlinkJarPath(); + + AbstractYarnClusterDescriptor yarnClusterDescriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(confProperties, flinkJarPath, launcherOptions.getQueue()); + ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph,true); + + String applicationId = clusterClient.getClusterId().toString(); + String flinkJobId = jobGraph.getJobID().toString(); + + String tips = String.format("deploy per_job with appId: %s, jobId: %s", applicationId, flinkJobId); + System.out.println(tips); + LOG.info(tips); + + return applicationId; + } +} From b4a2693a2f7371c03f63a10f661442fbef5766a0 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Sat, 17 Nov 2018 15:07:19 +0800 Subject: [PATCH 03/42] bugfix --- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 90e45d57e..080d3d48b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -106,7 +106,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; - jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); + joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } @@ -545,11 +545,11 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; - jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); + joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } } - private void jionFun(Object pollObj, Map localTableCache, + private void joinFun(Object pollObj, Map localTableCache, Map sideTableMap, StreamTableEnvironment tableEnv, List replaceInfoList) throws Exception{ JoinInfo joinInfo = (JoinInfo) pollObj; From c94f2bf08c52273cda85a93d4480fb2972a8b60c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 15:13:41 +0800 Subject: [PATCH 04/42] add per job submit mode --- README.md | 15 +++++++++++++-- .../perjob/PerJobClusterClientBuilder.java | 3 ++- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 3241a57a5..03ed36002 100644 --- a/README.md +++ b/README.md @@ -52,8 +52,9 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * **model** * 描述:执行模式,也就是flink集群的工作模式 * local: 本地模式 - * standalone: 独立部署模式的flink集群 - * yarn: yarn模式的flink集群 + * standalone: 提交到独立部署模式的flink集群 + * yarn: 提交到yarn模式的flink集群(即提交到已有flink集群) + * yarnPer: yarn per_job模式提交(即创建新flink application) * 必选:否 * 默认值:local @@ -118,6 +119,16 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:指示保存点是否允许非还原状态的标志 * 必选:否 * 默认值:false + +* **flinkJarPath** + * 描述:per_job 模式提交需要指定本地的flink jar存放路径 + * 必选:否 + * 默认值:false + +* **queue** + * 描述:per_job 模式下指定的yarn queue + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 7d395f0ac..820160219 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -77,7 +77,8 @@ public void init(String yarnConfDir){ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); - newConf.addAllToProperties(confProp); + //newConf.addAllToProperties(confProp); + confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString()) ); //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); From e9c870a3c4515a59ac7c48baef382e74a9331954 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:01:37 +0800 Subject: [PATCH 05/42] modify readme doc 1:add per_job mode start describe 2:add new metric describe --- README.md | 32 ++++++++++++++++++- docs/prometheus.md | 7 ++++ .../perjob/FLinkPerJobResourceUtil.java | 20 ++++++------ .../perjob/PerJobClusterClientBuilder.java | 30 ----------------- 4 files changed, 48 insertions(+), 41 deletions(-) create mode 100644 docs/prometheus.md diff --git a/README.md b/README.md index 03ed36002..50b4c27f0 100644 --- a/README.md +++ b/README.md @@ -98,6 +98,11 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// + * jobmanager.memory.mb: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:768) + * taskmanager.memory.mb: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:768) + * taskmanager.num: per_job模式下指定taskmanager的实例数(默认1) + * taskmanager.slots:per_job模式下指定每个taskmanager对应的slot数量(默认1) + * [prometheus 相关参数](docs/prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 * **flinkconf** @@ -146,8 +151,33 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mysql 维表插件](docs/mysqlSide.md) * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) + +## 3 性能指标(新增) + +### kafka插件 +* 业务延迟: flink_taskmanager_job_task_operator_dtEventDelay(单位s) + 数据本身的时间和进入flink的当前时间的差值. + +* 各个输入源的脏数据:flink_taskmanager_job_task_operator_dtDirtyData + 从kafka获取的数据解析失败的视为脏数据 + +* 各Source的数据输入TPS: flink_taskmanager_job_task_operator_dtNumRecordsInRate + kafka接受的记录数(未解析前)/s + +* 各Source的数据输入RPS: flink_taskmanager_job_task_operator_dtNumRecordsInResolveRate + kafka接受的记录数(解析后)/s + +* 各Source的数据输入BPS: flink_taskmanager_job_task_operator_dtNumBytesInRate + kafka接受的字节数/s + +* Kafka作为输入源的各个分区的延迟数: flink_taskmanager_job_task_operator_topic_partition_dtTopicPartitionLag + 当前kafka10,kafka11有采集该指标 + +* 各个输出源RPS: flink_taskmanager_job_task_operator_dtNumRecordsOutRate + 写入的外部记录数/s + -## 3 样例 +## 4 样例 ``` CREATE TABLE MyTable( diff --git a/docs/prometheus.md b/docs/prometheus.md new file mode 100644 index 000000000..a36498b42 --- /dev/null +++ b/docs/prometheus.md @@ -0,0 +1,7 @@ +## 使用 prometheus pushgateway 需要设置的 confProp 参数 +* metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter +* metrics.reporter.promgateway.host: prometheus pushgateway的地址 +* metrics.reporter.promgateway.port:prometheus pushgateway的端口 +* metrics.reporter.promgateway.jobName: 实例名称 +* metrics.reporter.promgateway.randomJobNameSuffix: 是否在实例名称后面添加随机字符串(默认:true) +* metrics.reporter.promgateway.deleteOnShutdown: 是否在停止的时候删除数据(默认false) \ No newline at end of file diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java index 3becc41d8..468039cb2 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java @@ -31,17 +31,17 @@ */ public class FLinkPerJobResourceUtil { - public final static int MIN_JM_MEMORY = 1024; // the minimum memory should be higher than the min heap cutoff - public final static int MIN_TM_MEMORY = 1024; + public final static int MIN_JM_MEMORY = 768; // the minimum memory should be higher than the min heap cutoff + public final static int MIN_TM_MEMORY = 768; public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; - public final static String CONTAINER = "container"; - public final static String SLOTS = "slots"; + public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; + public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; public static ClusterSpecification createClusterSpecification(Properties confProperties) { - int jobmanagerMemoryMb = 1024; - int taskmanagerMemoryMb = 1024; + int jobmanagerMemoryMb = 768; + int taskmanagerMemoryMb = 768; int numberTaskManagers = 1; int slotsPerTaskManager = 1; @@ -60,12 +60,12 @@ public static ClusterSpecification createClusterSpecification(Properties confPro } } - if (confProperties.containsKey(CONTAINER)){ - numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(CONTAINER)); + if (confProperties.containsKey(NUMBER_TASK_MANAGERS)){ + numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(NUMBER_TASK_MANAGERS)); } - if (confProperties.containsKey(SLOTS)){ - slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS)); + if (confProperties.containsKey(SLOTS_PER_TASKMANAGER)){ + slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS_PER_TASKMANAGER)); } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 820160219..8ddcd541d 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -44,20 +44,6 @@ public class PerJobClusterClientBuilder { - public static final String DEFAULT_GATEWAY_CLASS = "org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"; - - public static final String PROMGATEWAY_CLASS_KEY = "metrics.reporter.promgateway.class"; - - public static final String PROMGATEWAY_HOST_KEY = "metrics.reporter.promgateway.host"; - - public static final String PROMGATEWAY_PORT_KEY = "metrics.reporter.promgateway.port"; - - public static final String PROMGATEWAY_JOBNAME_KEY = "metrics.reporter.promgateway.jobName"; - - public static final String PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY = "metrics.reporter.promgateway.randomJobNameSuffix"; - - public static final String PROMGATEWAY_DELETEONSHUTDOWN_KEY = "metrics.reporter.promgateway.deleteOnShutdown"; - private YarnClient yarnClient; private YarnConfiguration yarnConf; @@ -77,10 +63,8 @@ public void init(String yarnConfDir){ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); - //newConf.addAllToProperties(confProp); confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString()) ); - //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); if (StringUtils.isNotBlank(flinkJarPath)) { @@ -115,20 +99,6 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co return clusterDescriptor; } - //FIXME need? - private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ - if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ - return; - } - - configuration.setString(PROMGATEWAY_CLASS_KEY, properties.getProperty(PROMGATEWAY_CLASS_KEY)); - configuration.setString(PROMGATEWAY_HOST_KEY, properties.getProperty(PROMGATEWAY_HOST_KEY)); - configuration.setString(PROMGATEWAY_PORT_KEY, properties.getProperty(PROMGATEWAY_PORT_KEY)); - configuration.setString(PROMGATEWAY_JOBNAME_KEY, properties.getProperty(PROMGATEWAY_JOBNAME_KEY)); - configuration.setString(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY, properties.getProperty(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY)); - configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); - } - private AbstractYarnClusterDescriptor getClusterDescriptor( Configuration configuration, YarnConfiguration yarnConfiguration, From c3a92d25684662c6832693d5d26005a380303307 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:37:14 +0800 Subject: [PATCH 06/42] exclude slf4j --- hbase/hbase-side/hbase-all-side/pom.xml | 2 +- hbase/hbase-side/hbase-async-side/pom.xml | 2 +- hbase/hbase-sink/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 6e344dd5d..7617a76e0 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -51,7 +51,7 @@ org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core - org.slf4j + org.slf4j:* diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 4da76c68c..193b720a0 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -52,7 +52,7 @@ org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core - org.slf4j + org.slf4j:* diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 9ec6d161c..26ead3574 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -27,7 +27,7 @@ - org.slf4j:slf4j-log4j12 + org.slf4j:* org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core From 53c8b20db1183afe271873a53a8435cd6d7ca8ec Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 14:08:21 +0800 Subject: [PATCH 07/42] support json --- launcher/pom.xml | 7 +++ .../flink/sql/launcher/LauncherMain.java | 45 ++++++++++++++++++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 927f6e2f3..2f8140fd5 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -29,6 +29,13 @@ log4j 1.2.17 + + + com.alibaba + fastjson + 1.2.7 + + diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 8e5832294..10bc09475 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -21,12 +21,18 @@ package com.dtstack.flink.sql.launcher; import avro.shaded.com.google.common.collect.Lists; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; -import java.io.File; + +import java.io.*; +import java.util.LinkedList; import java.util.List; +import java.util.Map; + import com.dtstack.flink.sql.ClusterMode; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; @@ -53,6 +59,9 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ } public static void main(String[] args) throws Exception { + if (args.length==1 && args[0].endsWith(".json")){ + args = parseJson(args); + } LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); @@ -85,4 +94,38 @@ public static void main(String[] args) throws Exception { System.out.println("---submit end----"); } + + private static String[] parseJson(String[] args) { + BufferedReader reader = null; + String lastStr = ""; + try{ + FileInputStream fileInputStream = new FileInputStream(args[0]); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream, "UTF-8"); + reader = new BufferedReader(inputStreamReader); + String tempString = null; + while((tempString = reader.readLine()) != null){ + lastStr += tempString; + } + reader.close(); + }catch(IOException e){ + e.printStackTrace(); + }finally{ + if(reader != null){ + try { + reader.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + Map map = JSON.parseObject(lastStr, new TypeReference>(){} ); + List list = new LinkedList<>(); + + for (Map.Entry entry : map.entrySet()) { + list.add("-" + entry.getKey()); + list.add(entry.getValue().toString()); + } + String[] array = list.toArray(new String[list.size()]); + return array; + } } From 277bedcbc7a5ff91c74649eee305db0366b449e4 Mon Sep 17 00:00:00 2001 From: zhihui-ge <2972333955@qq.com> Date: Mon, 19 Nov 2018 20:00:41 +0800 Subject: [PATCH 08/42] update kafka source --- docs/kafkaSource.md | 2 +- .../flink/sql/source/kafka/KafkaSource.java | 19 ++++++++++++++++++- .../flink/sql/source/kafka/KafkaSource.java | 19 ++++++++++++++++++- .../flink/sql/source/kafka/KafkaSource.java | 17 +++++++++++++++++ 4 files changed, 54 insertions(+), 3 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 2968dbe22..f382ba9ad 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -38,7 +38,7 @@ CREATE TABLE tableName( |bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| -|offsetReset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| ## 5.样例: diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8486b9883..00988518a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,16 +23,20 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -76,7 +80,20 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else{ + }else if(kafka09SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka09SourceTableInfo.getOffsetReset()); + } + }else { kafkaSrc.setStartFromLatest(); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index ffc42f7ef..c9eaf05bc 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,15 +23,19 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -75,7 +79,20 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else{ + }else if(kafka010SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); + } + }else { kafkaSrc.setStartFromLatest(); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8bfec8107..5f6146cc9 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,15 +23,19 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -75,6 +79,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); + }else if(kafka011SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); + } }else{ kafkaSrc.setStartFromLatest(); } From b138bac5551a3707a02957023566ae607a5162ad Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 22:13:05 +0800 Subject: [PATCH 09/42] fix jar conflict --- hbase/hbase-side/hbase-all-side/pom.xml | 4 ++++ hbase/pom.xml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 7617a76e0..32850176b 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -29,6 +29,10 @@ log4j log4j + + io.netty + netty + diff --git a/hbase/pom.xml b/hbase/pom.xml index 48c8162a0..835457e29 100644 --- a/hbase/pom.xml +++ b/hbase/pom.xml @@ -43,6 +43,10 @@ log4j log4j + + io.netty + netty + From 35f8567eddf602e4776c5af4b08705fe722d4763 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 20 Nov 2018 21:29:33 +0800 Subject: [PATCH 10/42] add init log info to hbase sink --- .../dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index a75f00da7..cc95d1e75 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.text.SimpleDateFormat; @@ -44,6 +46,8 @@ */ public class HbaseOutputFormat extends RichOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); + private String host; private String zkParent; private String[] rowkey; @@ -63,17 +67,21 @@ public class HbaseOutputFormat extends RichOutputFormat { @Override public void configure(Configuration parameters) { + LOG.warn("---configure---"); conf = HBaseConfiguration.create(); conf.set("hbase.zookeeper.quorum", host); if(zkParent != null && !"".equals(zkParent)){ conf.set("zookeeper.znode.parent", zkParent); } + LOG.warn("---configure end ---"); } @Override public void open(int taskNumber, int numTasks) throws IOException { + LOG.warn("---open---"); conn = ConnectionFactory.createConnection(conf); table = conn.getTable(TableName.valueOf(tableName)); + LOG.warn("---open end(get table from hbase) ---"); } @Override From 0af546ddef66bcdeb86698823534a885e1305a88 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 22 Nov 2018 10:21:08 +0800 Subject: [PATCH 11/42] TableInfoParser remove sourceTableInfoMap static --- .../java/com/dtstack/flink/sql/parser/SqlParser.java | 10 +++++----- ...ableInfoParserFactory.java => TableInfoParser.java} | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/table/{TableInfoParserFactory.java => TableInfoParser.java} (91%) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 4bd17c65e..7ab616632 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TableInfoParserFactory; +import com.dtstack.flink.sql.table.TableInfoParser; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; @@ -75,7 +75,7 @@ public static SqlTree parseSql(String sql) throws Exception { List sqlArr = DtStringUtil.splitIgnoreQuota(sql, SQL_DELIMITER); SqlTree sqlTree = new SqlTree(); - + TableInfoParser tableInfoParser = new TableInfoParser(); for(String childSql : sqlArr){ if(Strings.isNullOrEmpty(childSql)){ continue; @@ -112,7 +112,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -125,7 +125,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SINK.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SINK.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -141,7 +141,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java similarity index 91% rename from core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java rename to core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java index f74e7b620..b3a07d6d5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java @@ -42,7 +42,7 @@ * @author xuchao */ -public class TableInfoParserFactory { +public class TableInfoParser { private final static String TYPE_KEY = "type"; @@ -50,14 +50,14 @@ public class TableInfoParserFactory { private final static Pattern SIDE_PATTERN = Pattern.compile(SIDE_TABLE_SIGN); - private static Map sourceTableInfoMap = Maps.newConcurrentMap(); + private Map sourceTableInfoMap = Maps.newConcurrentMap(); - private static Map targetTableInfoMap = Maps.newConcurrentMap(); + private Map targetTableInfoMap = Maps.newConcurrentMap(); - private static Map sideTableInfoMap = Maps.newConcurrentMap(); + private Map sideTableInfoMap = Maps.newConcurrentMap(); //Parsing loaded plugin - public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, + public TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, String localPluginRoot) throws Exception { AbsTableParser absTableParser = null; Map props = parserResult.getPropMap(); From 3b5298c99186c3a79daea963a567e8d5ec118c7a Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 10:57:36 +0800 Subject: [PATCH 12/42] fix hbase thread bug --- .../com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 75c65b73b..33dd9b9a9 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -85,10 +85,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List(), new DTThreadFactory("hbase-aysnc")); - hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); + hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent()); try { Deferred deferred = hBaseClient.ensureTableExists(tableName) From 88f4287e9e7bda0644b541b63dc662a3e2d402d4 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 16:02:16 +0800 Subject: [PATCH 13/42] support the pool configuration of redis --- .../sql/sink/redis/RedisOutputFormat.java | 110 +++++++++++++----- .../flink/sql/sink/redis/RedisSink.java | 24 +++- .../sql/sink/redis/table/RedisSinkParser.java | 9 +- .../sql/sink/redis/table/RedisTableInfo.java | 79 +++++++++++-- 4 files changed, 182 insertions(+), 40 deletions(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 742b2a377..90131d34a 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -39,6 +39,16 @@ public class RedisOutputFormat extends RichOutputFormat { private String password; + private int redisType; + + private String maxTotal; + + private String maxIdle; + + private String minIdle; + + private String masterName; + protected String[] fieldNames; protected TypeInformation[] fieldTypes; @@ -53,6 +63,8 @@ public class RedisOutputFormat extends RichOutputFormat { private JedisSentinelPool jedisSentinelPool; + private JedisCluster jedisCluster; + private GenericObjectPoolConfig poolConfig; private RedisOutputFormat(){ @@ -67,26 +79,48 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); } + private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ + GenericObjectPoolConfig config = new GenericObjectPoolConfig(); + if (maxTotal != null){ + config.setMaxTotal(Integer.parseInt(maxTotal)); + } + if (maxIdle != null){ + config.setMaxIdle(Integer.parseInt(maxIdle)); + } + if (minIdle != null){ + config.setMinIdle(Integer.parseInt(minIdle)); + } + return config; + } + private void establishConnection() { - poolConfig = new GenericObjectPoolConfig(); + poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); - if (nodes.length > 1){ - //cluster - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = ipPort.split(":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); - } - jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); - jedis = jedisSentinelPool.getResource(); - } else { - String[] ipPortPair = nodes[0].split(":"); - String ip = ipPortPair[0]; - String port = ipPortPair[1]; - pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), timeout, password, Integer.parseInt(database)); - jedis = pool.getResource(); + String[] firstIpPort = nodes[0].split(":"); + String firstIp = firstIpPort[0]; + String firstPort = firstIpPort[1]; + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + + switch (redisType){ + //单机 + case 1: + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + break; + //哨兵 + case 2: + jedisSentinelPool = new JedisSentinelPool(masterName, ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + jedis = jedisSentinelPool.getResource(); + break; + //集群 + case 3: + jedisCluster = new JedisCluster(addresses, timeout, timeout,1, poolConfig); } } @@ -126,7 +160,12 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - jedis.set(key.toString(), (String) row.getField(i)); + if (redisType != 3){ + jedis.set(key.toString(), (String) row.getField(i)); + } else { + jedisCluster.set(key.toString(), (String) row.getField(i)); + } + } } @@ -192,23 +231,40 @@ public RedisOutputFormatBuilder setTimeout(int timeout){ return this; } + public RedisOutputFormatBuilder setRedisType(int redisType){ + redisOutputFormat.redisType = redisType; + return this; + } + + public RedisOutputFormatBuilder setMaxTotal(String maxTotal){ + redisOutputFormat.maxTotal = maxTotal; + return this; + } + + public RedisOutputFormatBuilder setMaxIdle(String maxIdle){ + redisOutputFormat.maxIdle = maxIdle; + return this; + } + + public RedisOutputFormatBuilder setMinIdle(String minIdle){ + redisOutputFormat.minIdle = minIdle; + return this; + } + + public RedisOutputFormatBuilder setMasterName(String masterName){ + redisOutputFormat.masterName = masterName; + return this; + } + public RedisOutputFormat finish(){ if (redisOutputFormat.url == null){ throw new IllegalArgumentException("No URL supplied."); } - if (redisOutputFormat.database == null){ - throw new IllegalArgumentException("No database supplied."); - } - if (redisOutputFormat.tableName == null){ throw new IllegalArgumentException("No tablename supplied."); } - if (redisOutputFormat.password == null){ - throw new IllegalArgumentException("No password supplied."); - } - return redisOutputFormat; } } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java index c11d96b8e..3e7300929 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -52,6 +52,16 @@ public class RedisSink implements RetractStreamTableSink, IStreamSinkGener< protected int timeout; + protected int redisType; + + protected String maxTotal; + + protected String maxIdle; + + protected String minIdle; + + protected String masterName; + public RedisSink(){ } @@ -63,7 +73,12 @@ public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { this.database = redisTableInfo.getDatabase(); this.password = redisTableInfo.getPassword(); this.tableName = redisTableInfo.getTablename(); - this.primaryKeys = targetTableInfo.getPrimaryKeys(); + this.primaryKeys = redisTableInfo.getPrimaryKeys(); + this.redisType = redisTableInfo.getRedisType(); + this.maxTotal = redisTableInfo.getMaxTotal(); + this.maxIdle = redisTableInfo.getMaxIdle(); + this.minIdle = redisTableInfo.getMinIdle(); + this.masterName = redisTableInfo.getMasterName(); return this; } @@ -82,7 +97,12 @@ public void emitDataStream(DataStream> dataStream) { .setFieldNames(this.fieldNames) .setFieldTypes(this.fieldTypes) .setPrimaryKeys(this.primaryKeys) - .setTimeout(this.timeout); + .setTimeout(this.timeout) + .setRedisType(this.redisType) + .setMaxTotal(this.maxTotal) + .setMaxIdle(this.maxIdle) + .setMinIdle(this.minIdle) + .setMasterName(this.masterName); RedisOutputFormat redisOutputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(redisOutputFormat); dataStream.addSink(richSinkFunction); diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 1bd74eb65..cecf70f95 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -34,9 +34,12 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Thu, 22 Nov 2018 16:11:32 +0800 Subject: [PATCH 14/42] add metric --- .../flink/sql/sink/redis/RedisOutputFormat.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 90131d34a..9e16a9b8f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,11 +18,15 @@ package com.dtstack.flink.sql.sink.redis; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import redis.clients.jedis.*; @@ -67,6 +71,10 @@ public class RedisOutputFormat extends RichOutputFormat { private GenericObjectPoolConfig poolConfig; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + private RedisOutputFormat(){ } @Override @@ -77,6 +85,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); + initMetric(); } private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ @@ -93,6 +102,11 @@ private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, S return config; } + private void initMetric(){ + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + private void establishConnection() { poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); @@ -165,8 +179,8 @@ public void writeRecord(Tuple2 record) throws IOException { } else { jedisCluster.set(key.toString(), (String) row.getField(i)); } - } + outRecords.inc(); } @Override From 5da0748aef12ba962680790bbcc6a94a009f03dc Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 22 Nov 2018 17:17:28 +0800 Subject: [PATCH 15/42] modify HBaseClient init ThreadPoolExecutor pool size --- .../flink/sql/side/hbase/HbaseAsyncReqRow.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 33dd9b9a9..fe227bb6a 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -63,7 +63,12 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(HbaseAsyncReqRow.class); - private static final int HBASE_WORKER_POOL_SIZE = 10; + //match to the rule of netty3 + private static final int DEFAULT_BOSS_THREADS = 1; + + private static final int DEFAULT_IO_THREADS = Runtime.getRuntime().availableProcessors() * 2; + + private static final int DEFAULT_POOL_SIZE = DEFAULT_IO_THREADS + DEFAULT_BOSS_THREADS; private transient HBaseClient hBaseClient; @@ -85,7 +90,11 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List(), new DTThreadFactory("hbase-aysnc")); + + hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); try { Deferred deferred = hBaseClient.ensureTableExists(tableName) From d00f5b45f4daae4e25bf5f7455ea3334ef697601 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 21:50:15 +0800 Subject: [PATCH 16/42] support the pool configuration of redis --- .../flink/sql/side/redis/RedisAllReqRow.java | 162 +++++++++++++----- .../sql/side/redis/RedisAsyncReqRow.java | 63 +++++-- .../sql/side/redis/RedisAsyncSideInfo.java | 1 - .../sql/side/redis/table/RedisSideParser.java | 9 +- .../side/redis/table/RedisSideTableInfo.java | 75 +++++++- .../sql/sink/redis/RedisOutputFormat.java | 21 ++- .../flink/sql/sink/redis/RedisSink.java | 1 + .../sql/sink/redis/table/RedisSinkParser.java | 4 +- .../sql/sink/redis/table/RedisTableInfo.java | 2 +- 9 files changed, 266 insertions(+), 72 deletions(-) diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 5b9ebbe10..e0e0a7d17 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -31,6 +31,8 @@ import org.slf4j.LoggerFactory; import redis.clients.jedis.*; +import java.io.Closeable; +import java.io.IOException; import java.sql.SQLException; import java.sql.Timestamp; import java.util.*; @@ -45,8 +47,6 @@ public class RedisAllReqRow extends AllReqRow{ private static final int CONN_RETRY_NUM = 3; - private static final int TIMEOUT = 10000; - private JedisPool pool; private JedisSentinelPool jedisSentinelPool; @@ -147,13 +147,13 @@ private String buildKey(Map inputParams) { } private void loadData(Map> tmpCache) throws SQLException { - Jedis jedis = null; + JedisCommands jedis = null; try { for(int i=0; i> tmpCache) throws SQLExcep } } - String perKey = tableInfo.getTableName() + "*"; - Set keys = jedis.keys(perKey); - List newPerKeys = new LinkedList<>(); - for (String key : keys){ - String[] splitKey = key.split(":"); - String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; - newPerKeys.add(newKey); - } - List list = newPerKeys.stream().distinct().collect(Collectors.toList()); - for(String key : list){ - Map kv = Maps.newHashMap(); - String[] primaryKv = key.split(":"); - kv.put(primaryKv[1], primaryKv[2]); - - String pattern = key + "*"; - Set realKeys = jedis.keys(pattern); - for (String realKey : realKeys){ - kv.put(realKey.split(":")[3], jedis.get(realKey)); + if (tableInfo.getRedisType() != 3){ + String perKey = tableInfo.getTableName() + "*"; + Set keys = ((Jedis) jedis).keys(perKey); + List newPerKeys = new LinkedList<>(); + for (String key : keys){ + String[] splitKey = key.split(":"); + String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; + newPerKeys.add(newKey); + } + List list = newPerKeys.stream().distinct().collect(Collectors.toList()); + for(String key : list){ + Map kv = Maps.newHashMap(); + String[] primaryKv = key.split(":"); + kv.put(primaryKv[1], primaryKv[2]); + String pattern = key + "*"; + Set realKeys = ((Jedis) jedis).keys(pattern); + for (String realKey : realKeys){ + kv.put(realKey.split(":")[3], jedis.get(realKey)); + } + tmpCache.put(key, kv); + } + } else { + String perKey = tableInfo.getTableName() + "*"; + Set keys = keys((JedisCluster) jedis, perKey); + List newPerKeys = new LinkedList<>(); + for (String key : keys){ + String[] splitKey = key.split(":"); + String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; + newPerKeys.add(newKey); + } + List list = newPerKeys.stream().distinct().collect(Collectors.toList()); + for(String key : list){ + Map kv = Maps.newHashMap(); + String[] primaryKv = key.split(":"); + kv.put(primaryKv[1], primaryKv[2]); + String pattern = key + "*"; + Set realKeys = keys((JedisCluster) jedis, pattern); + for (String realKey : realKeys){ + kv.put(realKey.split(":")[3], jedis.get(realKey)); + } + tmpCache.put(key, kv); } - tmpCache.put(key, kv); } @@ -197,7 +219,11 @@ private void loadData(Map> tmpCache) throws SQLExcep LOG.error("", e); } finally { if (jedis != null){ - jedis.close(); + try { + ((Closeable) jedis).close(); + } catch (IOException e) { + e.printStackTrace(); + } } if (jedisSentinelPool != null) { jedisSentinelPool.close(); @@ -208,26 +234,78 @@ private void loadData(Map> tmpCache) throws SQLExcep } } - private Jedis getJedis(String url, String password, String database){ - JedisPoolConfig poolConfig = new JedisPoolConfig(); + private JedisCommands getJedis(RedisSideTableInfo tableInfo) { + String url = tableInfo.getUrl(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + int timeout = tableInfo.getTimeout(); + if (timeout == 0){ + timeout = 1000; + } + String[] nodes = url.split(","); - if (nodes.length > 1){ - //cluster - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = ipPort.split(":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + String[] firstIpPort = nodes[0].split(":"); + String firstIp = firstIpPort[0]; + String firstPort = firstIpPort[1]; + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + if (timeout == 0){ + timeout = 1000; + } + JedisCommands jedis = null; + GenericObjectPoolConfig poolConfig = setPoolConfig(tableInfo.getMaxTotal(), tableInfo.getMaxIdle(), tableInfo.getMinIdle()); + switch (tableInfo.getRedisType()){ + //单机 + case 1: + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + break; + //哨兵 + case 2: + jedisSentinelPool = new JedisSentinelPool(tableInfo.getMasterName(), ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + jedis = jedisSentinelPool.getResource(); + break; + //集群 + case 3: + jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + } + + return jedis; + } + + private Set keys(JedisCluster jedisCluster, String pattern){ + Set keys = new TreeSet<>(); + Map clusterNodes = jedisCluster.getClusterNodes(); + for(String k : clusterNodes.keySet()){ + JedisPool jp = clusterNodes.get(k); + Jedis connection = jp.getResource(); + try { + keys.addAll(connection.keys(pattern)); + } catch (Exception e){ + LOG.error("Getting keys error: {}", e); + } finally { + connection.close(); } - jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, TIMEOUT, password, Integer.parseInt(database)); - return jedisSentinelPool.getResource(); - } else { - String[] ipPortPair = nodes[0].split(":"); - String ip = ipPortPair[0]; - String port = ipPortPair[1]; - pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), TIMEOUT, password, Integer.parseInt(database)); - return pool.getResource(); } + return keys; + } + + private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ + GenericObjectPoolConfig config = new GenericObjectPoolConfig(); + if (maxTotal != null){ + config.setMaxTotal(Integer.parseInt(maxTotal)); + } + if (maxIdle != null){ + config.setMaxIdle(Integer.parseInt(maxIdle)); + } + if (minIdle != null){ + config.setMinIdle(Integer.parseInt(minIdle)); + } + return config; } } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index a541c9300..82532c527 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -22,12 +22,14 @@ import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; -import io.lettuce.core.KeyScanCursor; import io.lettuce.core.KeyValue; import io.lettuce.core.RedisClient; import io.lettuce.core.RedisFuture; import io.lettuce.core.api.StatefulRedisConnection; -import io.lettuce.core.api.async.RedisAsyncCommands; +import io.lettuce.core.api.async.RedisKeyAsyncCommands; +import io.lettuce.core.api.async.RedisStringAsyncCommands; +import io.lettuce.core.cluster.RedisClusterClient; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; @@ -40,9 +42,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; import java.util.function.Consumer; -import java.util.stream.Collectors; public class RedisAsyncReqRow extends AsyncReqRow { @@ -52,7 +52,11 @@ public class RedisAsyncReqRow extends AsyncReqRow { private StatefulRedisConnection connection; - private RedisAsyncCommands async; + private RedisClusterClient clusterClient; + + private StatefulRedisClusterConnection clusterConnection; + + private RedisKeyAsyncCommands async; private RedisSideTableInfo redisSideTableInfo; @@ -65,19 +69,42 @@ public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List 1){ - uri.append("redis-sentinel://").append(password).append("@") - .append(url).append("/").append(database).append("#").append(url.split(",")[0]); - } else { - uri.append("redis://").append(password).append("@").append(url).append("/").append(database); + if (database == null){ + database = "0"; + } + switch (tableInfo.getRedisType()){ + case 1: + StringBuilder redisUri = new StringBuilder(); + redisUri.append("redis://").append(password).append(url).append("/").append(database); + redisClient = RedisClient.create(redisUri.toString()); + connection = redisClient.connect(); + async = connection.async(); + break; + case 2: + StringBuilder sentinelUri = new StringBuilder(); + sentinelUri.append("redis-sentinel://").append(password) + .append(url).append("/").append(database).append("#").append(redisSideTableInfo.getMasterName()); + redisClient = RedisClient.create(sentinelUri.toString()); + connection = redisClient.connect(); + async = connection.async(); + break; + case 3: + StringBuilder clusterUri = new StringBuilder(); + clusterUri.append("redis://").append(password).append(url); + clusterClient = RedisClusterClient.create(clusterUri.toString()); + clusterConnection = clusterClient.connect(); + async = clusterConnection.async(); } - redisClient = RedisClient.create(uri.toString()); - connection = redisClient.connect(); - async = connection.async(); } @Override @@ -143,7 +170,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Map keyValue = Maps.newHashMap(); List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); - RedisFuture>> future = async.mget(values); + RedisFuture>> future = ((RedisStringAsyncCommands) async).mget(values); future.thenAccept(new Consumer>>() { @Override public void accept(List> keyValues) { @@ -185,6 +212,12 @@ public void close() throws Exception { if (redisClient != null){ redisClient.shutdown(); } + if (clusterConnection != null){ + clusterConnection.close(); + } + if (clusterClient != null){ + clusterClient.shutdown(); + } } } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index a5cf0fcaf..a019b74cf 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -22,7 +22,6 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index d9a56d892..fc0d3cb6f 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -47,7 +47,14 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private JedisPool pool; - private Jedis jedis; + private JedisCommands jedis; private JedisSentinelPool jedisSentinelPool; - private JedisCluster jedisCluster; - private GenericObjectPoolConfig poolConfig; private transient Counter outRecords; @@ -120,6 +119,9 @@ private void establishConnection() { String[] ipPortPair = ipPort.split(":"); addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); } + if (timeout == 0){ + timeout = 10000; + } switch (redisType){ //单机 @@ -134,7 +136,7 @@ private void establishConnection() { break; //集群 case 3: - jedisCluster = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); } } @@ -174,11 +176,7 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - if (redisType != 3){ - jedis.set(key.toString(), (String) row.getField(i)); - } else { - jedisCluster.set(key.toString(), (String) row.getField(i)); - } + jedis.set(key.toString(), (String) row.getField(i)); } outRecords.inc(); } @@ -191,6 +189,11 @@ public void close() throws IOException { if (pool != null) { pool.close(); } + if (jedis != null){ + if (jedis instanceof Closeable){ + ((Closeable) jedis).close(); + } + } } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java index 3e7300929..d2e28c01f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -79,6 +79,7 @@ public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { this.maxIdle = redisTableInfo.getMaxIdle(); this.minIdle = redisTableInfo.getMinIdle(); this.masterName = redisTableInfo.getMasterName(); + this.timeout = redisTableInfo.getTimeout(); return this; } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index cecf70f95..7b1633257 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -34,7 +34,9 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Fri, 23 Nov 2018 16:47:30 +0800 Subject: [PATCH 17/42] fix mysql output bug(type of float) --- .../dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index f09b2a1eb..4a231b043 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -226,9 +226,9 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce pstmt.setLong(index + 1, (long) row.getField(index)); break; case java.sql.Types.REAL: + case java.sql.Types.FLOAT: pstmt.setFloat(index + 1, (float) row.getField(index)); break; - case java.sql.Types.FLOAT: case java.sql.Types.DOUBLE: pstmt.setDouble(index + 1, (double) row.getField(index)); break; From 57c64d9e93b53d8dd81ade9f55081d687e695a8e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 23 Nov 2018 17:01:45 +0800 Subject: [PATCH 18/42] comment --- .../sink/mysql/RetractJDBCOutputFormat.java | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index f09b2a1eb..74e100417 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -16,26 +16,6 @@ * limitations under the License. */ - - -/* - * 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 com.dtstack.flink.sql.sink.mysql; import com.dtstack.flink.sql.metric.MetricConstant; From eb90a4b21a04836f4cb30e137d07869126ab68cd Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 23 Nov 2018 17:17:10 +0800 Subject: [PATCH 19/42] add rdb module and add sqlserver side all request --- pom.xml | 1 + rdb/pom.xml | 40 ++++ rdb/sqlserver/pom.xml | 40 ++++ rdb/sqlserver/sqlserver-side/pom.xml | 25 ++ .../sqlserver-side/sqlserver-all-side/pom.xml | 87 +++++++ .../side/sqlserver/SqlserverAllReqRow.java | 221 ++++++++++++++++++ .../side/sqlserver/SqlserverAllSideInfo.java | 89 +++++++ .../sqlserver-side-core/pom.xml | 18 ++ .../sqlserver/table/SqlserverSideParser.java | 41 ++++ .../table/SqlserverSideTableInfo.java | 69 ++++++ 10 files changed, 631 insertions(+) create mode 100644 rdb/pom.xml create mode 100644 rdb/sqlserver/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java diff --git a/pom.xml b/pom.xml index 07694589c..f6911ad58 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,7 @@ mongo redis5 launcher + rdb pom diff --git a/rdb/pom.xml b/rdb/pom.xml new file mode 100644 index 000000000..6b1a7fd8a --- /dev/null +++ b/rdb/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.rdb + 1.0-SNAPSHOT + + sqlserver + + pom + + + 3.8.1 + + + + + junit + junit + ${junit.version} + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/pom.xml b/rdb/sqlserver/pom.xml new file mode 100644 index 000000000..beb9bb8aa --- /dev/null +++ b/rdb/sqlserver/pom.xml @@ -0,0 +1,40 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sqlserver + 1.0-SNAPSHOT + pom + + + 1.3.1 + + + + sqlserver-side + + + + + net.sourceforge.jtds + jtds + ${jtds.version} + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/pom.xml b/rdb/sqlserver/sqlserver-side/pom.xml new file mode 100644 index 000000000..dc9229931 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/pom.xml @@ -0,0 +1,25 @@ + + + + sql.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.sqlserver + 1.0-SNAPSHOT + + sqlserver-side + pom + + + sqlserver-side-core + sqlserver-all-side + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml new file mode 100644 index 000000000..45f36422e --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -0,0 +1,87 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.sqlserver + sqlserver-all-side + 1.0-SNAPSHOT + jar + + + + com.dtstack.flink + sql.side.sqlserver.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java new file mode 100644 index 000000000..5ef274377 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -0,0 +1,221 @@ +package com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + */ +public class SqlserverAllReqRow extends AllReqRow { + + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); + + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); + } + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } else { + return; + } + + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + + } + + /** + * load data by diff db + * + * @param tmpCache + */ + private void loadData(Map>> tmpCache) throws SQLException { + //这个地方抽取为RdbSideTableInfo + SqlserverSideTableInfo tableInfo = (SqlserverSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + + try { + connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + + try { + String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + + } + + //load data from table + String sql = sideInfo.getSqlCondition(); + Statement statement = connection.createStatement(); + statement.setFetchSize(FETCH_SIZE); + ResultSet resultSet = statement.executeQuery(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + while (resultSet.next()) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } catch (Exception e) { + LOG.error("", e); + } finally { + if (connection != null) { + connection.close(); + } + } + } + + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + /** + * 这个方法创建在RdbAllReqRow.java中 + * + * @param dbURL + * @param userName + * @param password + * @return + */ + private Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(SQLSERVER_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + //addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + + + +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java new file mode 100644 index 000000000..f112ae251 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -0,0 +1,89 @@ +package com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +//import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +public class SqlserverAllSideInfo extends SideInfo { + + + public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + SqlserverSideTableInfo sqlserverSideTableInfo = (SqlserverSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", sqlserverSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + + } + + @Override + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + if(CollectionUtils.isEmpty(equalFieldList)){ + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for(String equalField : equalFieldList){ + if(fields.contains(equalField)){ + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml new file mode 100644 index 000000000..6cb64eb7e --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -0,0 +1,18 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.sqlserver.core + 1.0-SNAPSHOT + jar + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java new file mode 100644 index 000000000..4fbe35c17 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -0,0 +1,41 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class SqlserverSideParser extends AbsSideTableParser { + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, SqlserverSideParser::dealSideSign); + } + + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + SqlserverSideTableInfo sqlserverSideTableInfo = new SqlserverSideTableInfo(); + sqlserverSideTableInfo.setName(tableName); + + parseFieldsInfo(fieldsInfo, sqlserverSideTableInfo); + parseCacheProp(sqlserverSideTableInfo, props); + + sqlserverSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(SqlserverSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + sqlserverSideTableInfo.setUrl(MathUtil.getString(props.get(SqlserverSideTableInfo.URL_KEY.toLowerCase()))); + sqlserverSideTableInfo.setTableName(MathUtil.getString(props.get(SqlserverSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + sqlserverSideTableInfo.setUserName(MathUtil.getString(props.get(SqlserverSideTableInfo.USER_NAME_KEY.toLowerCase()))); + sqlserverSideTableInfo.setPassword(MathUtil.getString(props.get(SqlserverSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return sqlserverSideTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo) { + } +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java new file mode 100644 index 000000000..f8f41d9d4 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java @@ -0,0 +1,69 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.side.SideTableInfo; + + +public class SqlserverSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -1L; + + private static final String CURR_TYPE = "sqlserver"; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public SqlserverSideTableInfo() { + setType(CURR_TYPE); + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + @Override + public boolean check() { + return false; + } + +} From 154cf4cd8d3e25a679ed085009d8aae06f775162 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 18:03:57 +0800 Subject: [PATCH 20/42] fix mysql output bug(type of float) --- .../main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 0fe9426c5..c2a4e50fd 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -113,7 +113,9 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.CHAR; }else if(fieldType.equals(Byte.class.getName())){ tmpFieldsType[i] = Types.BINARY; - }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ + }else if(fieldType.equals(Float.class.getName())){ + tmpFieldsType[i] = Types.FLOAT; + }else if(fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; }else if (fieldType.equals(Timestamp.class.getName())){ tmpFieldsType[i] = Types.TIMESTAMP; From 86af0ec838569446699f5a209be60de2d33b54a9 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 21:31:32 +0800 Subject: [PATCH 21/42] fix mysql output bug(type of BigDecimal) --- .../main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index c2a4e50fd..c3526ecd1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.math.BigDecimal; import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -117,8 +118,10 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.FLOAT; }else if(fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; - }else if (fieldType.equals(Timestamp.class.getName())){ + }else if(fieldType.equals(Timestamp.class.getName())){ tmpFieldsType[i] = Types.TIMESTAMP; + }else if(fieldType.equals(BigDecimal.class.getName())){ + tmpFieldsType[i] = Types.DECIMAL; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 3be45371565cf13b3bed676c2136b7104be3e953 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 26 Nov 2018 11:02:00 +0800 Subject: [PATCH 22/42] move side sign parser to absclass --- .../flink/sql/table/AbsSideTableParser.java | 15 ++++++++++++++ .../sql/side/hbase/table/HbaseSideParser.java | 10 ---------- .../sql/side/mongo/table/MongoSideParser.java | 12 ----------- .../sql/side/mysql/table/MysqlSideParser.java | 12 ----------- .../sqlserver/table/SqlserverSideParser.java | 13 ------------ .../sql/side/redis/table/RedisSideParser.java | 20 +++++-------------- 6 files changed, 20 insertions(+), 62 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index e46a72494..f8ede801b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -25,6 +25,8 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +37,19 @@ public abstract class AbsSideTableParser extends AbsTableParser { + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, AbsSideTableParser::dealSideSign); + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + //FIXME SIDE_TABLE_SIGN current just used as a sign for side table; and do nothing + } + //Analytical create table attributes ==> Get information cache protected void parseCacheProp(SideTableInfo sideTableInfo, Map props){ if(props.containsKey(SideTableInfo.CACHE_KEY.toLowerCase())){ diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 299db961d..3031de100 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -44,10 +44,6 @@ public class HbaseSideParser extends AbsSideTableParser { private final static Pattern FIELD_PATTERN = Pattern.compile("(?i)(.*)\\s+AS\\s+(\\w+)$"); - private final static String SIDE_SIGN_KEY = "sideSignKey"; - - private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); - public static final String HBASE_ZOOKEEPER_QUORUM = "zookeeperQuorum"; public static final String ZOOKEEPER_PARENT = "zookeeperParent"; @@ -60,9 +56,6 @@ public class HbaseSideParser extends AbsSideTableParser { static { - keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); - keyHandlerMap.put(SIDE_SIGN_KEY, HbaseSideParser::dealSideSign); - keyPatternMap.put(FIELD_KEY, FIELD_PATTERN); keyHandlerMap.put(FIELD_KEY, HbaseSideParser::dealField); } @@ -82,9 +75,6 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MongoSideTableInfo mongoSideTableInfo = new MongoSideTableInfo(); @@ -73,7 +64,4 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MysqlSideTableInfo mysqlTableInfo = new MysqlSideTableInfo(); @@ -61,7 +52,4 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RedisSideTableInfo redisSideTableInfo = new RedisSideTableInfo(); redisSideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, redisSideTableInfo); @@ -47,17 +37,17 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 26 Nov 2018 16:07:29 +0800 Subject: [PATCH 23/42] youhua --- .../dtstack/flink/sql/util/DtStringUtil.java | 18 ++++++++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- 4 files changed, 24 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index db74480d6..959d4b13c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -21,8 +21,10 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.enums.ColumnType; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.math.BigDecimal; import java.util.ArrayList; @@ -42,6 +44,9 @@ public class DtStringUtil { private static final Pattern NO_VERSION_PATTERN = Pattern.compile("([a-zA-Z]+).*"); + private static ObjectMapper objectMapper = new ObjectMapper(); + + /** * Split the specified string delimiter --- ignored quotes delimiter * @param str @@ -207,4 +212,17 @@ public static String addJdbcParam(String dbUrl, Map addParams, b return preStr + "?" + sb.toString(); } + + public static boolean isJosn(String str){ + boolean flag = false; + if(StringUtils.isNotBlank(str)){ + try { + objectMapper.readValue(str,Map.class); + flag = true; + } catch (Throwable e) { + flag=false; + } + } + return flag; + } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 00988518a..cbc697c82 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; @@ -80,9 +81,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka09SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka09SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index c9eaf05bc..2953d0f86 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -79,9 +80,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka010SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 5f6146cc9..d10151920 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -79,9 +80,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka011SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); From 96c56176f10abc524bb790b101a0dcaa43d31c86 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 11:12:29 +0800 Subject: [PATCH 24/42] create rdb module and extract common code --- pom.xml | 76 +++--- rdb/pom.xml | 3 +- rdb/rdb-side/pom.xml | 17 ++ rdb/rdb-sink/pom.xml | 17 ++ .../side/sqlserver/SqlserverAllReqRow.java | 221 ------------------ .../side/sqlserver/SqlserverAllSideInfo.java | 89 ------- .../table/SqlserverSideTableInfo.java | 69 ------ {rdb/sqlserver => sqlserver}/pom.xml | 12 +- .../sqlserver-side/pom.xml | 4 +- .../sqlserver-side/sqlserver-all-side/pom.xml | 4 +- .../sqlserver-side-core/pom.xml | 1 - 11 files changed, 83 insertions(+), 430 deletions(-) create mode 100644 rdb/rdb-side/pom.xml create mode 100644 rdb/rdb-sink/pom.xml delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java rename {rdb/sqlserver => sqlserver}/pom.xml (88%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/pom.xml (93%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/sqlserver-all-side/pom.xml (98%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/sqlserver-side-core/pom.xml (92%) diff --git a/pom.xml b/pom.xml index f6911ad58..808b55976 100644 --- a/pom.xml +++ b/pom.xml @@ -1,43 +1,45 @@ - 4.0.0 + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + 4.0.0 - com.dtstack.flink - flink.sql - 1.0-SNAPSHOT - - core - kafka09 - kafka10 - kafka11 - mysql - hbase - elasticsearch5 - mongo - redis5 - launcher - rdb - - pom + com.dtstack.flink + flink.sql + 1.0-SNAPSHOT + + core + kafka09 + kafka10 + kafka11 + mysql + hbase + elasticsearch5 + mongo + redis5 + launcher + rdb + sqlserver + rdb + + pom - flink.sql - http://maven.apache.org + flink.sql + http://maven.apache.org - - UTF-8 - 1.5.4 - + + UTF-8 + 1.5.4 + - - - - org.apache.maven.plugins - maven-compiler-plugin - - 1.8 - 1.8 - - - - + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + diff --git a/rdb/pom.xml b/rdb/pom.xml index 6b1a7fd8a..705ad728c 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -12,7 +12,8 @@ sql.rdb 1.0-SNAPSHOT - sqlserver + rdb-side + rdb-sink pom diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml new file mode 100644 index 000000000..0103ca4cf --- /dev/null +++ b/rdb/rdb-side/pom.xml @@ -0,0 +1,17 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.rdb + 1.0-SNAPSHOT + rdb-side + jar + + \ No newline at end of file diff --git a/rdb/rdb-sink/pom.xml b/rdb/rdb-sink/pom.xml new file mode 100644 index 000000000..6b4868c95 --- /dev/null +++ b/rdb/rdb-sink/pom.xml @@ -0,0 +1,17 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.sink.rdb + 1.0-SNAPSHOT + rdb-sink + jar + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java deleted file mode 100644 index 5ef274377..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ /dev/null @@ -1,221 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver; - -import com.dtstack.flink.sql.side.*; -import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.calcite.sql.JoinType; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.*; -import java.util.Calendar; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; - -/** - * side operator with cache for all(period reload) - */ -public class SqlserverAllReqRow extends AllReqRow { - - - private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); - - private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - - private static final int CONN_RETRY_NUM = 3; - - private static final int FETCH_SIZE = 1000; - - private AtomicReference>>> cacheRef = new AtomicReference<>(); - - public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); - } - - @Override - protected Row fillData(Row input, Object sideInput) { - Map cacheInfo = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. - if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { - obj = ((Timestamp) obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { - if (cacheInfo == null) { - row.setField(entry.getKey(), null); - } else { - row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); - } - } - - return row; - } - - @Override - protected void initCache() throws SQLException { - Map>> newCache = Maps.newConcurrentMap(); - cacheRef.set(newCache); - loadData(newCache); - } - - @Override - protected void reloadCache() { - //reload cacheRef and replace to old cacheRef - Map>> newCache = Maps.newConcurrentMap(); - try { - loadData(newCache); - } catch (SQLException e) { - LOG.error("", e); - } - - cacheRef.set(newCache); - LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); - } - - @Override - public void flatMap(Row value, Collector out) throws Exception { - List inputParams = Lists.newArrayList(); - for (Integer conValIndex : sideInfo.getEqualValIndex()) { - Object equalObj = value.getField(conValIndex); - if (equalObj == null) { - out.collect(null); - } - - inputParams.add(equalObj); - } - - String key = buildKey(inputParams); - List> cacheList = cacheRef.get().get(key); - if (CollectionUtils.isEmpty(cacheList)) { - if (sideInfo.getJoinType() == JoinType.LEFT) { - Row row = fillData(value, null); - out.collect(row); - } else { - return; - } - - return; - } - - for (Map one : cacheList) { - out.collect(fillData(value, one)); - } - - } - - /** - * load data by diff db - * - * @param tmpCache - */ - private void loadData(Map>> tmpCache) throws SQLException { - //这个地方抽取为RdbSideTableInfo - SqlserverSideTableInfo tableInfo = (SqlserverSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; - - - try { - for (int i = 0; i < CONN_RETRY_NUM; i++) { - - try { - connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); - break; - } catch (Exception e) { - if (i == CONN_RETRY_NUM - 1) { - throw new RuntimeException("", e); - } - - try { - String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); - LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); - } catch (InterruptedException e1) { - e1.printStackTrace(); - } - } - - } - - //load data from table - String sql = sideInfo.getSqlCondition(); - Statement statement = connection.createStatement(); - statement.setFetchSize(FETCH_SIZE); - ResultSet resultSet = statement.executeQuery(sql); - String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); - while (resultSet.next()) { - Map oneRow = Maps.newHashMap(); - for (String fieldName : sideFieldNames) { - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); - } - - String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); - List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); - list.add(oneRow); - } - } catch (Exception e) { - LOG.error("", e); - } finally { - if (connection != null) { - connection.close(); - } - } - } - - - private String buildKey(List equalValList) { - StringBuilder sb = new StringBuilder(""); - for (Object equalVal : equalValList) { - sb.append(equalVal).append("_"); - } - return sb.toString(); - } - - private String buildKey(Map val, List equalFieldList) { - StringBuilder sb = new StringBuilder(""); - for (String equalField : equalFieldList) { - sb.append(val.get(equalField)).append("_"); - } - - return sb.toString(); - } - - /** - * 这个方法创建在RdbAllReqRow.java中 - * - * @param dbURL - * @param userName - * @param password - * @return - */ - private Connection getConn(String dbURL, String userName, String password) { - try { - Class.forName(SQLSERVER_DRIVER); - //add param useCursorFetch=true - Map addParams = Maps.newHashMap(); - //addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); - return DriverManager.getConnection(targetDbUrl, userName, password); - } catch (Exception e) { - LOG.error("", e); - throw new RuntimeException("", e); - } - } - - - -} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java deleted file mode 100644 index f112ae251..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java +++ /dev/null @@ -1,89 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver; - -import com.dtstack.flink.sql.side.FieldInfo; -import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -//import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; - -import java.util.List; - -public class SqlserverAllSideInfo extends SideInfo { - - - public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - SqlserverSideTableInfo sqlserverSideTableInfo = (SqlserverSideTableInfo) sideTableInfo; - - sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", sqlserverSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); - - } - - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - if(CollectionUtils.isEmpty(equalFieldList)){ - throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); - } - - for(String equalField : equalFieldList){ - if(fields.contains(equalField)){ - continue; - } - - fields.add(equalField); - } - - sideSelectFields = String.join(",", fields); - } -} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java deleted file mode 100644 index f8f41d9d4..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java +++ /dev/null @@ -1,69 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver.table; - -import com.dtstack.flink.sql.side.SideTableInfo; - - -public class SqlserverSideTableInfo extends SideTableInfo { - - private static final long serialVersionUID = -1L; - - private static final String CURR_TYPE = "sqlserver"; - - public static final String URL_KEY = "url"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; - - public SqlserverSideTableInfo() { - setType(CURR_TYPE); - } - - private String url; - - private String tableName; - - private String userName; - - private String password; - - public String getUrl() { - return url; - } - - public void setUrl(String url) { - this.url = url; - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } - - @Override - public boolean check() { - return false; - } - -} diff --git a/rdb/sqlserver/pom.xml b/sqlserver/pom.xml similarity index 88% rename from rdb/sqlserver/pom.xml rename to sqlserver/pom.xml index beb9bb8aa..05f880a3f 100644 --- a/rdb/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -3,25 +3,23 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - sql.rdb + flink.sql com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 sql.sqlserver 1.0-SNAPSHOT + + sqlserver-side + pom 1.3.1 - - sqlserver-side - - net.sourceforge.jtds @@ -31,7 +29,7 @@ com.dtstack.flink - sql.core + sql.side.rdb 1.0-SNAPSHOT provided diff --git a/rdb/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml similarity index 93% rename from rdb/sqlserver/sqlserver-side/pom.xml rename to sqlserver/sqlserver-side/pom.xml index dc9229931..fc9179f62 100644 --- a/rdb/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -6,7 +6,6 @@ sql.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 @@ -19,7 +18,6 @@ sqlserver-side-core sqlserver-all-side + sqlserver-side-core - - \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml similarity index 98% rename from rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml rename to sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index 45f36422e..a11abe0f4 100644 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -6,13 +6,12 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 sql.side.all.sqlserver - sqlserver-all-side 1.0-SNAPSHOT + sqlserver-all-side jar @@ -84,4 +83,5 @@ + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml similarity index 92% rename from rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml rename to sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index 6cb64eb7e..36940fe66 100644 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -6,7 +6,6 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 From 251440bdb530b19b03a91f0207be4e670ace1614 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:42:33 +0800 Subject: [PATCH 25/42] sqlserver side depend on rdb --- pom.xml | 1 - rdb/pom.xml | 3 +- rdb/rdb-side/pom.xml | 23 +++++++++++++++ .../sqlserver/table/SqlserverSideParser.java | 28 ------------------- sqlserver/pom.xml | 19 +++++++++---- sqlserver/sqlserver-side/pom.xml | 20 ++++++++++++- .../sqlserver-side/sqlserver-all-side/pom.xml | 12 +++++--- .../sqlserver-side-core/pom.xml | 1 + 8 files changed, 67 insertions(+), 40 deletions(-) delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java diff --git a/pom.xml b/pom.xml index 808b55976..bde73d6df 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,6 @@ launcher rdb sqlserver - rdb pom diff --git a/rdb/pom.xml b/rdb/pom.xml index 705ad728c..0ef3aa252 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -19,6 +19,7 @@ 3.8.1 + 1.0-SNAPSHOT @@ -32,7 +33,7 @@ com.dtstack.flink sql.core - 1.0-SNAPSHOT + ${sql.core.version} provided diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml index 0103ca4cf..2c1680fb0 100644 --- a/rdb/rdb-side/pom.xml +++ b/rdb/rdb-side/pom.xml @@ -14,4 +14,27 @@ rdb-side jar + + 3.5.2 + + + + + + + + io.vertx + vertx-jdbc-client + ${vertx.version} + + + + io.vertx + vertx-core + ${vertx.version} + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java deleted file mode 100644 index 7ad2d6f0c..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver.table; - -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; - -import java.util.Map; - -public class SqlserverSideParser extends AbsSideTableParser { - - - @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - SqlserverSideTableInfo sqlserverSideTableInfo = new SqlserverSideTableInfo(); - sqlserverSideTableInfo.setName(tableName); - - parseFieldsInfo(fieldsInfo, sqlserverSideTableInfo); - parseCacheProp(sqlserverSideTableInfo, props); - - sqlserverSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(SqlserverSideTableInfo.PARALLELISM_KEY.toLowerCase()))); - sqlserverSideTableInfo.setUrl(MathUtil.getString(props.get(SqlserverSideTableInfo.URL_KEY.toLowerCase()))); - sqlserverSideTableInfo.setTableName(MathUtil.getString(props.get(SqlserverSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); - sqlserverSideTableInfo.setUserName(MathUtil.getString(props.get(SqlserverSideTableInfo.USER_NAME_KEY.toLowerCase()))); - sqlserverSideTableInfo.setPassword(MathUtil.getString(props.get(SqlserverSideTableInfo.PASSWORD_KEY.toLowerCase()))); - - return sqlserverSideTableInfo; - } -} diff --git a/sqlserver/pom.xml b/sqlserver/pom.xml index 05f880a3f..463fd55c8 100644 --- a/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -18,21 +18,30 @@ 1.3.1 + 1.0-SNAPSHOT + 1.0-SNAPSHOT - net.sourceforge.jtds - jtds - ${jtds.version} + com.dtstack.flink + sql.side.rdb + ${sql.side.rdb.version} + provided com.dtstack.flink - sql.side.rdb - 1.0-SNAPSHOT + sql.core + ${sql.core.version} provided + + + net.sourceforge.jtds + jtds + ${jtds.version} + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml index fc9179f62..dac53bed0 100644 --- a/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -15,9 +15,27 @@ sqlserver-side pom + + 1.0-SNAPSHOT + + + sqlserver-side-core sqlserver-all-side - sqlserver-side-core + sqlserver-async-side + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index a11abe0f4..b188aaedf 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -14,11 +14,15 @@ sqlserver-all-side jar + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.sqlserver.core - 1.0-SNAPSHOT + ${sql.side.sqlserver.core.version} @@ -68,14 +72,14 @@ - + - + diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index 36940fe66..aed72e25f 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -14,4 +14,5 @@ jar + \ No newline at end of file From 742312ae7ef4d9ba5154e6f85972ee8bdb559171 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:50:30 +0800 Subject: [PATCH 26/42] sqlserver side depend on rdb module --- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 219 ++++++++++++++++++ .../sql/side/rdb/all/RdbAllSideInfo.java | 115 +++++++++ .../sql/side/rdb/async/RdbAsyncReqRow.java | 195 ++++++++++++++++ .../sql/side/rdb/async/RdbAsyncSideInfo.java | 135 +++++++++++ .../sql/side/rdb/table/RdbSideParser.java | 53 +++++ .../sql/side/rdb/table/RdbSideTableInfo.java | 72 ++++++ .../side/sqlserver/SqlserverAllReqRow.java | 47 ++++ .../side/sqlserver/SqlserverAllSideInfo.java | 16 ++ .../sqlserver-async-side/pom.xml | 91 ++++++++ .../side/sqlserver/SqlserverAsyncReqRow.java | 71 ++++++ .../sqlserver/SqlserverAsyncSideInfo.java | 35 +++ .../sqlserver/table/SqlserverSideParser.java | 17 ++ 12 files changed, 1066 insertions(+) create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java create mode 100644 sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/pom.xml create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java new file mode 100644 index 000000000..118ee048f --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -0,0 +1,219 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.all; + +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public abstract class RdbAllReqRow extends AllReqRow { + + private static final long serialVersionUID = 2098635140857937718L; + + private static final Logger LOG = LoggerFactory.getLogger(RdbAllReqRow.class); + + //private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RdbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } else { + return; + } + + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + + } + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + public abstract Connection getConn(String dbURL, String userName, String password); + + + private void loadData(Map>> tmpCache) throws SQLException { + RdbSideTableInfo tableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + + try { + connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + + try { + String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + + } + + //load data from table + String sql = sideInfo.getSqlCondition(); + Statement statement = connection.createStatement(); + statement.setFetchSize(FETCH_SIZE); + ResultSet resultSet = statement.executeQuery(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + while (resultSet.next()) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } catch (Exception e) { + LOG.error("", e); + } finally { + if (connection != null) { + connection.close(); + } + } + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java new file mode 100644 index 000000000..501686f95 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -0,0 +1,115 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.all; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAllSideInfo extends SideInfo { + + private static final long serialVersionUID = -5858335638589472159L; + + + public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo) { + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for (int i = 0; i < outFieldInfoList.size(); i++) { + FieldInfo fieldInfo = outFieldInfoList.get(i); + if (fieldInfo.getTable().equalsIgnoreCase(sideTableName)) { + fields.add(fieldInfo.getFieldName()); + sideFieldIndex.put(i, sideIndex); + sideFieldNameIndex.put(i, fieldInfo.getFieldName()); + sideIndex++; + } else if (fieldInfo.getTable().equalsIgnoreCase(nonSideTableName)) { + int nonSideIndex = rowTypeInfo.getFieldIndex(fieldInfo.getFieldName()); + inFieldIndex.put(i, nonSideIndex); + } else { + throw new RuntimeException("unknown table " + fieldInfo.getTable()); + } + } + + if (fields.size() == 0) { + throw new RuntimeException("select non field from table " + sideTableName); + } + + //add join on condition field to select fields + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + if (CollectionUtils.isEmpty(equalFieldList)) { + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for (String equalField : equalFieldList) { + if (fields.contains(equalField)) { + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java new file mode 100644 index 000000000..15f1ee923 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -0,0 +1,195 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.async; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.cache.CacheObj; +import io.vertx.core.json.JsonArray; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 2098635244857937720L; + + private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); + + private transient SQLClient rdbSQLClient; + + public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RdbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + +// @Override +// public void open(Configuration parameters) throws Exception{ +// super.open(parameters); +// } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = input.getField(conValIndex); + if (equalObj == null) { + resultFuture.complete(null); + } + + inputParams.add(equalObj); + } + + String key = buildCacheKey(inputParams); + if (openCache()) { + CacheObj val = getFromCache(key); + if (val != null) { + + if (ECacheContentType.MissVal == val.getType()) { + dealMissKey(input, resultFuture); + return; + } else if (ECacheContentType.MultiLine == val.getType()) { + + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, jsonArray); + resultFuture.complete(Collections.singleton(row)); + } + + } else { + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + rdbSQLClient.getConnection(conn -> { + if (conn.failed()) { + //Treatment failures + resultFuture.completeExceptionally(conn.cause()); + return; + } + + final SQLConnection connection = conn.result(); + String sqlCondition = sideInfo.getSqlCondition(); + connection.queryWithParams(sqlCondition, inputParams, rs -> { + if (rs.failed()) { + LOG.error("Cannot retrieve the data from the database"); + LOG.error("", rs.cause()); + resultFuture.complete(null); + return; + } + + List cacheContent = Lists.newArrayList(); + + int resultSize = rs.result().getResults().size(); + if (resultSize > 0) { + for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input, line); + if (openCache()) { + cacheContent.add(line); + } + resultFuture.complete(Collections.singleton(row)); + } + + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + // and close the connection + connection.close(done -> { + if (done.failed()) { + throw new RuntimeException(done.cause()); + } + }); + }); + }); + } + + @Override + public Row fillData(Row input, Object line) { + JsonArray jsonArray = (JsonArray) line; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()) { + if (jsonArray == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + } + } + + return row; + } + + @Override + public void close() throws Exception { + super.close(); + rdbSQLClient.close(); + } + + public String buildCacheKey(JsonArray jsonArray) { + StringBuilder sb = new StringBuilder(); + for (Object ele : jsonArray.getList()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + + public void setRdbSQLClient(SQLClient rdbSQLClient) { + this.rdbSQLClient = rdbSQLClient; + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java new file mode 100644 index 000000000..b7ff94ea9 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -0,0 +1,135 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.async; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAsyncSideInfo extends SideInfo { + + public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for (int i = 0; i < equalFieldList.size(); i++) { + String equalField = equalFieldList.get(i); + + sqlCondition += equalField + "=? "; + if (i != equalFieldList.size() - 1) { + sqlCondition += " and "; + } + } + + sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + + @Override + public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { + if (sqlNode.getKind() != SqlKind.EQUALS) { + throw new RuntimeException("not equal operator."); + } + + SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlIdentifier right = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[1]; + + String leftTableName = left.getComponent(0).getSimple(); + String leftField = left.getComponent(1).getSimple(); + + String rightTableName = right.getComponent(0).getSimple(); + String rightField = right.getComponent(1).getSimple(); + + if (leftTableName.equalsIgnoreCase(sideTableName)) { + equalFieldList.add(leftField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(rightField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode); + } + + equalValIndex.add(equalFieldIndex); + + } else if (rightTableName.equalsIgnoreCase(sideTableName)) { + + equalFieldList.add(rightField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(leftField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode.toString()); + } + + equalValIndex.add(equalFieldIndex); + + } else { + throw new RuntimeException("resolve equalFieldList error:" + sqlNode.toString()); + } + + } + +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java new file mode 100644 index 000000000..82d013a98 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -0,0 +1,53 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbSideParser extends AbsSideTableParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RdbSideTableInfo rdbTableInfo = new RdbSideTableInfo(); + rdbTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, rdbTableInfo); + + parseCacheProp(rdbTableInfo, props); + rdbTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(RdbSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + rdbTableInfo.setUrl(MathUtil.getString(props.get(RdbSideTableInfo.URL_KEY.toLowerCase()))); + rdbTableInfo.setTableName(MathUtil.getString(props.get(RdbSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + rdbTableInfo.setUserName(MathUtil.getString(props.get(RdbSideTableInfo.USER_NAME_KEY.toLowerCase()))); + rdbTableInfo.setPassword(MathUtil.getString(props.get(RdbSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return rdbTableInfo; + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java new file mode 100644 index 000000000..2117639c4 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -0,0 +1,72 @@ +package com.dtstack.flink.sql.side.rdb.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ +public class RdbSideTableInfo extends SideTableInfo { + private static final long serialVersionUID = -1L; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "rdb of URL is required"); + Preconditions.checkNotNull(tableName, "rdb of tableName is required"); + Preconditions.checkNotNull(userName, "rdb of userName is required"); + Preconditions.checkNotNull(password, "rdb of password is required"); + return true; + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java new file mode 100644 index 000000000..ef2786f57 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -0,0 +1,47 @@ +package com.dtstack.flink.sql.side.sqlserver; + + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.List; +import java.util.Map; + +/** + * side operator with cache for all(period reload) + */ +public class SqlserverAllReqRow extends RdbAllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); + + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(SQLSERVER_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + //addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + +} diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java new file mode 100644 index 000000000..f671dc0e6 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -0,0 +1,16 @@ +package com.dtstack.flink.sql.side.sqlserver; + + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import java.util.List; + +public class SqlserverAllSideInfo extends RdbAllSideInfo { + + public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml new file mode 100644 index 000000000..08413dc43 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml @@ -0,0 +1,91 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.sqlserver + 1.0-SNAPSHOT + sqlserver-async-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.sqlserver.core + ${sql.side.sqlserver.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java new file mode 100644 index 000000000..1774d4388 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -0,0 +1,71 @@ +/* + * 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 com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.jdbc.JDBCClient; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + + +public class SqlserverAsyncReqRow extends RdbAsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAsyncReqRow.class); + + private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; + + private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; + + private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + + public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject sqlserverClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + sqlserverClientConfig.put("url", rdbSideTableInfo.getUrl()) + .put("driver_class", SQLSERVER_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + Vertx vertx = Vertx.vertx(vo); + setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java new file mode 100644 index 000000000..347ca9bf4 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java @@ -0,0 +1,35 @@ +/* + * 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 com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import java.util.List; + + + +public class SqlserverAsyncSideInfo extends RdbAsyncSideInfo { + + public SqlserverAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java new file mode 100644 index 000000000..edd4820be --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -0,0 +1,17 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; +import java.util.Map; + + +public class SqlserverSideParser extends RdbSideParser { + private static final String CURR_TYPE = "sqlserver"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlServerTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlServerTableInfo.setType(CURR_TYPE); + return sqlServerTableInfo; + } +} From 3cea7224cdaee072128c47e83841b11be807f1cc Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 26 Nov 2018 21:59:15 +0800 Subject: [PATCH 27/42] pom.xml --- rdb/rdb-side/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml index 2c1680fb0..c7350877f 100644 --- a/rdb/rdb-side/pom.xml +++ b/rdb/rdb-side/pom.xml @@ -6,6 +6,7 @@ sql.rdb com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 From 34f28c5f619af789033f045027e20754e1ffbfb6 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 27 Nov 2018 20:37:12 +0800 Subject: [PATCH 28/42] mysql module depend on rdb --- mysql/mysql-side/mysql-all-side/pom.xml | 10 +- .../flink/sql/side/mysql/MysqlAllReqRow.java | 186 +-------- .../sql/side/mysql/MysqlAllSideInfo.java | 78 +--- mysql/mysql-side/mysql-async-side/pom.xml | 25 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 166 +------- .../sql/side/mysql/MysqlAsyncSideInfo.java | 99 +---- mysql/mysql-side/mysql-side-core/pom.xml | 10 +- .../sql/side/mysql/table/MysqlSideParser.java | 24 +- .../side/mysql/table/MysqlSideTableInfo.java | 99 ----- mysql/mysql-side/pom.xml | 17 +- mysql/mysql-sink/pom.xml | 14 +- .../flink/sql/sink/mysql/MysqlSink.java | 53 +-- .../sink/mysql/RetractJDBCOutputFormat.java | 362 ------------------ .../sql/sink/mysql/table/MysqlSinkParser.java | 23 +- mysql/pom.xml | 13 +- rdb/pom.xml | 1 + .../sql/side/rdb/async/RdbAsyncReqRow.java | 9 +- .../sql/side/rdb/table/RdbSideTableInfo.java | 17 + rdb/rdb-sink/pom.xml | 1 + .../dtstack/flink/sql/sink/rdb}/DBSink.java | 68 ++-- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 71 ++++ .../sql/sink/rdb/RetractJDBCOutputFormat.java | 362 ++++++++++++++++++ .../sql/sink/rdb/table/RdbSinkParser.java | 51 +++ .../sql/sink/rdb/table/RdbTableInfo.java | 35 +- sqlserver/pom.xml | 17 +- sqlserver/sqlserver-side/pom.xml | 4 +- .../sqlserver-side/sqlserver-all-side/pom.xml | 1 + .../side/sqlserver/SqlserverAllReqRow.java | 17 + .../side/sqlserver/SqlserverAllSideInfo.java | 17 + .../sqlserver-async-side/pom.xml | 1 + .../side/sqlserver/SqlserverAsyncReqRow.java | 1 - .../sqlserver-side-core/pom.xml | 3 +- .../sqlserver/table/SqlserverSideParser.java | 17 + sqlserver/sqlserver-sink/pom.xml | 92 +++++ .../sql/sink/sqlserver/SqlserverSink.java | 47 +++ .../sqlserver/table/SqlserverSinkParser.java | 41 ++ 36 files changed, 904 insertions(+), 1148 deletions(-) delete mode 100644 mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java delete mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java rename {mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql => rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb}/DBSink.java (79%) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java rename mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java => rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java (81%) create mode 100644 sqlserver/sqlserver-sink/pom.xml create mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java create mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java diff --git a/mysql/mysql-side/mysql-all-side/pom.xml b/mysql/mysql-side/mysql-all-side/pom.xml index 4d6feb236..92ce13a85 100644 --- a/mysql/mysql-side/mysql-all-side/pom.xml +++ b/mysql/mysql-side/mysql-all-side/pom.xml @@ -15,11 +15,15 @@ jar + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.mysql.core - 1.0-SNAPSHOT + ${sql.side.mysql.core.version} @@ -71,12 +75,12 @@ - + + tofile="${basedir}/../../../plugins/mysqlallside/${project.name}.jar"/> diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index a8839e951..fa4895ded 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -18,42 +18,30 @@ package com.dtstack.flink.sql.side.mysql; -import com.dtstack.flink.sql.side.AllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.calcite.sql.JoinType; -import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Timestamp; -import java.util.Calendar; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; /** * side operator with cache for all(period reload) * Date: 2018/9/19 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAllReqRow extends AllReqRow{ +public class MysqlAllReqRow extends RdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -61,176 +49,22 @@ public class MysqlAllReqRow extends AllReqRow{ private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; - - private static final int FETCH_SIZE = 1000; - - private AtomicReference>>> cacheRef = new AtomicReference<>(); - - public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo){ - super(new MysqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); - } - - @Override - protected Row fillData(Row input, Object sideInput) { - Map cacheInfo = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ - if(cacheInfo == null){ - row.setField(entry.getKey(), null); - }else{ - row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); - } - } - - return row; - } - - @Override - protected void initCache() throws SQLException { - Map>> newCache = Maps.newConcurrentMap(); - cacheRef.set(newCache); - loadData(newCache); + public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - protected void reloadCache() { - //reload cacheRef and replace to old cacheRef - Map>> newCache = Maps.newConcurrentMap(); + public Connection getConn(String dbURL, String userName, String password) { try { - loadData(newCache); - } catch (SQLException e) { - LOG.error("", e); - } - - cacheRef.set(newCache); - LOG.info("----- mysql all cacheRef reload end:{}", Calendar.getInstance()); - } - - - @Override - public void flatMap(Row value, Collector out) throws Exception { - List inputParams = Lists.newArrayList(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ - Object equalObj = value.getField(conValIndex); - if(equalObj == null){ - out.collect(null); - } - - inputParams.add(equalObj); - } - - String key = buildKey(inputParams); - List> cacheList = cacheRef.get().get(key); - if(CollectionUtils.isEmpty(cacheList)){ - if(sideInfo.getJoinType() == JoinType.LEFT){ - Row row = fillData(value, null); - out.collect(row); - }else{ - return; - } - - return; - } - - for(Map one : cacheList){ - out.collect(fillData(value, one)); - } - - } - - private String buildKey(List equalValList){ - StringBuilder sb = new StringBuilder(""); - for(Object equalVal : equalValList){ - sb.append(equalVal).append("_"); - } - - return sb.toString(); - } - - private String buildKey(Map val, List equalFieldList){ - StringBuilder sb = new StringBuilder(""); - for(String equalField : equalFieldList){ - sb.append(val.get(equalField)).append("_"); - } - - return sb.toString(); - } - - private Connection getConn(String dbURL, String userName, String password){ - try{ Class.forName(MYSQL_DRIVER); //add param useCursorFetch=true - Map addParams = Maps.newHashMap(); + Map addParams = Maps.newHashMap(); addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams,true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); - }catch (Exception e){ + } catch (Exception e) { LOG.error("", e); throw new RuntimeException("", e); } } - - - private void loadData(Map>> tmpCache) throws SQLException { - MysqlSideTableInfo tableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; - - try{ - for(int i=0; i oneRow = Maps.newHashMap(); - for(String fieldName : sideFieldNames){ - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); - } - - String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); - List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); - list.add(oneRow); - } - }catch (Exception e){ - LOG.error("", e); - }finally { - if(connection != null){ - connection.close(); - } - } - } } diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java index 2364aaac8..81193c0f9 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -20,15 +20,9 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.commons.collections.CollectionUtils; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.util.List; @@ -36,78 +30,12 @@ * Reason: * Date: 2018/9/19 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAllSideInfo extends SideInfo{ - - private static final long serialVersionUID = -5858335638589472158L; - +public class MysqlAllSideInfo extends RdbAllSideInfo { public MysqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; - - sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", mysqlSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); - } - - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - if(CollectionUtils.isEmpty(equalFieldList)){ - throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); - } - - for(String equalField : equalFieldList){ - if(fields.contains(equalField)){ - continue; - } - - fields.add(equalField); - } - - sideSelectFields = String.join(",", fields); - } } diff --git a/mysql/mysql-side/mysql-async-side/pom.xml b/mysql/mysql-side/mysql-async-side/pom.xml index a7cf1a70e..a5588ea97 100644 --- a/mysql/mysql-side/mysql-async-side/pom.xml +++ b/mysql/mysql-side/mysql-async-side/pom.xml @@ -16,27 +16,16 @@ jar - - - - - io.vertx - vertx-jdbc-client - 3.5.2 - - - - io.vertx - vertx-core - 3.5.2 - + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.mysql.core - 1.0-SNAPSHOT + ${sql.side.mysql.core.version} - @@ -87,12 +76,12 @@ - + + tofile="${basedir}/../../../plugins/mysqlasyncside/${project.name}.jar"/> diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index f72df5dea..86fe8c8fa 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -16,55 +16,37 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mysql; -import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; -import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.cache.CacheObj; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; -import io.vertx.core.json.JsonArray; import io.vertx.core.json.JsonObject; import io.vertx.ext.jdbc.JDBCClient; -import io.vertx.ext.sql.SQLClient; -import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.Timestamp; -import java.util.Collections; import java.util.List; -import java.util.Map; /** * Mysql dim table * Date: 2018/7/27 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAsyncReqRow extends AsyncReqRow { - - private static final long serialVersionUID = 2098635244857937717L; +public class MysqlAsyncReqRow extends RdbAsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(MysqlAsyncReqRow.class); - private transient SQLClient mySQLClient; - private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; @@ -75,152 +57,26 @@ public class MysqlAsyncReqRow extends AsyncReqRow { public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new MysqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - JsonObject mySQLClientConfig = new JsonObject(); - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); - mySQLClientConfig.put("url", mysqlSideTableInfo.getUrl()) + JsonObject mysqlClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + mysqlClientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", MYSQL_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) - .put("user", mysqlSideTableInfo.getUserName()) - .put("password", mysqlSideTableInfo.getPassword()); + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); Vertx vertx = Vertx.vertx(vo); - mySQLClient = JDBCClient.createNonShared(vertx, mySQLClientConfig); - } - - @Override - public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { - - JsonArray inputParams = new JsonArray(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ - Object equalObj = input.getField(conValIndex); - if(equalObj == null){ - resultFuture.complete(null); - } - - inputParams.add(equalObj); - } - - String key = buildCacheKey(inputParams); - if(openCache()){ - CacheObj val = getFromCache(key); - if(val != null){ - - if(ECacheContentType.MissVal == val.getType()){ - dealMissKey(input, resultFuture); - return; - }else if(ECacheContentType.MultiLine == val.getType()){ - - for(Object jsonArray : (List)val.getContent()){ - Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); - } - - }else{ - throw new RuntimeException("not support cache obj type " + val.getType()); - } - return; - } - } - - mySQLClient.getConnection(conn -> { - if (conn.failed()) { - //Treatment failures - resultFuture.completeExceptionally(conn.cause()); - return; - } - - final SQLConnection connection = conn.result(); - String sqlCondition = sideInfo.getSqlCondition(); - connection.queryWithParams(sqlCondition, inputParams, rs -> { - if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); - resultFuture.complete(null); - return; - } - - List cacheContent = Lists.newArrayList(); - - int resultSize = rs.result().getResults().size(); - if(resultSize > 0){ - for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); - if(openCache()){ - cacheContent.add(line); - } - resultFuture.complete(Collections.singleton(row)); - } - - if(openCache()){ - putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); - } - }else{ - dealMissKey(input, resultFuture); - if(openCache()){ - putCache(key, CacheMissVal.getMissKeyObj()); - } - } - - // and close the connection - connection.close(done -> { - if (done.failed()) { - throw new RuntimeException(done.cause()); - } - }); - }); - }); - } - - @Override - public Row fillData(Row input, Object line){ - JsonArray jsonArray = (JsonArray) line; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(jsonArray == null){ - row.setField(entry.getKey(), null); - }else{ - row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); - } - } - - return row; - } - - @Override - public void close() throws Exception { - super.close(); - mySQLClient.close(); - } - - public String buildCacheKey(JsonArray jsonArray){ - StringBuilder sb = new StringBuilder(); - for(Object ele : jsonArray.getList()){ - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); + setRdbSQLClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); } } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java index 18f1220d9..f72671ce0 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java @@ -20,15 +20,9 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.util.List; @@ -40,98 +34,9 @@ * @author xuchao */ -public class MysqlAsyncSideInfo extends SideInfo { - - private static final long serialVersionUID = -5931494270201575201L; +public class MysqlAsyncSideInfo extends RdbAsyncSideInfo { public MysqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; - - String sideTableName = joinInfo.getSideTableName(); - - SqlNode conditionNode = joinInfo.getCondition(); - - List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - sqlCondition = "select ${selectField} from ${tableName} where "; - for(int i=0; i4.0.0 sql.side.mysql.core - - - com.dtstack.flink - sql.core - 1.0-SNAPSHOT - provided - - + 1.0-SNAPSHOT jar + mysql-side-core \ No newline at end of file diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java index 94a50e8ab..40f68e7e4 100644 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java +++ b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java @@ -16,40 +16,30 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mysql.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * Reason: * Date: 2018/7/25 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSideParser extends AbsSideTableParser { +public class MysqlSideParser extends RdbSideParser { + + private static final String CURR_TYPE = "mysql"; @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MysqlSideTableInfo mysqlTableInfo = new MysqlSideTableInfo(); - mysqlTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, mysqlTableInfo); - - parseCacheProp(mysqlTableInfo, props); - mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlSideTableInfo.PARALLELISM_KEY.toLowerCase()))); - mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlSideTableInfo.URL_KEY.toLowerCase()))); - mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlSideTableInfo.USER_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlSideTableInfo.PASSWORD_KEY.toLowerCase()))); - + TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } } diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java deleted file mode 100644 index 72ac0dc55..000000000 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.side.mysql.table; - -import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; - -/** - * Reason: - * Date: 2018/7/25 - * Company: www.dtstack.com - * @author xuchao - */ - -public class MysqlSideTableInfo extends SideTableInfo { - - private static final long serialVersionUID = -1L; - - private static final String CURR_TYPE = "mysql"; - - public static final String URL_KEY = "url"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; - - public MysqlSideTableInfo(){ - setType(CURR_TYPE); - } - - @Override - public boolean check() { - Preconditions.checkNotNull(url, "mysql of URL is required"); - Preconditions.checkNotNull(tableName, "mysql of tableName is required"); - Preconditions.checkNotNull(userName, "mysql of userName is required"); - Preconditions.checkNotNull(password, "mysql of password is required"); - return true; - } - - private String url; - - private String tableName; - - private String userName; - - private String password; - - public String getUrl() { - return url; - } - - public void setUrl(String url) { - this.url = url; - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } -} diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index 78957c1e0..4f104b07f 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -11,13 +11,28 @@ 4.0.0 sql.side.mysql + 1.0-SNAPSHOT mysql-side + pom + mysql-side-core mysql-async-side mysql-all-side - pom + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + \ No newline at end of file diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 1aeeccba6..01b964da8 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -14,8 +14,16 @@ mysql-sink http://maven.apache.org - + + 1.0-SNAPSHOT + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + @@ -66,12 +74,12 @@ - + + tofile="${basedir}/../../plugins/mysqlsink/${project.name}.jar"/> diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index ba6eeb7f2..fe190f10f 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -16,40 +16,39 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.mysql; -import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.sink.mysql.table.MysqlTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.sink.rdb.RdbSink; -import java.util.Arrays; import java.util.List; /** * Date: 2017/2/27 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSink extends DBSink implements IStreamSinkGener { +public class MysqlSink extends RdbSink { + + private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - public MysqlSink(){ + public MysqlSink() { } @Override - public void buildSql(String tableName, List fields){ + public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); } - private void buildInsertSql(String tableName, List fields){ + private void buildInsertSql(String tableName, List fields) { String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; String fieldsStr = ""; String placeholder = ""; - for(String fieldName : fields){ + for (String fieldName : fields) { fieldsStr += ",`" + fieldName + "`"; placeholder += ",?"; } @@ -61,38 +60,10 @@ private void buildInsertSql(String tableName, List fields){ this.sql = sqlTmp; } - @Override - public MysqlSink genStreamSink(TargetTableInfo targetTableInfo) { - - MysqlTableInfo mysqlTableInfo = (MysqlTableInfo) targetTableInfo; - - String tmpDbURL = mysqlTableInfo.getUrl(); - String tmpUserName = mysqlTableInfo.getUserName(); - String tmpPassword = mysqlTableInfo.getPassword(); - String tmpTableName = mysqlTableInfo.getTableName(); - Integer tmpSqlBatchSize = mysqlTableInfo.getBatchSize(); - if(tmpSqlBatchSize != null){ - setBatchInterval(tmpSqlBatchSize); - } - - Integer tmpSinkParallelism = mysqlTableInfo.getParallelism(); - if(tmpSinkParallelism != null){ - setParallelism(tmpSinkParallelism); - } - - List fields = Arrays.asList(mysqlTableInfo.getFields()); - List fieldTypeArray = Arrays.asList(mysqlTableInfo.getFieldClasses()); - - this.driverName = "com.mysql.jdbc.Driver"; - this.dbURL = tmpDbURL; - this.userName = tmpUserName; - this.password = tmpPassword; - this.tableName = tmpTableName; - this.primaryKeys = mysqlTableInfo.getPrimaryKeys(); - buildSql(tableName, fields); - buildSqlTypes(fieldTypeArray); - return this; + @Override + public String getDriverName() { + return MYSQL_DRIVER; } } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java deleted file mode 100644 index 8ad543ae8..000000000 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ /dev/null @@ -1,362 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.sink.mysql; - -import com.dtstack.flink.sql.metric.MetricConstant; -import org.apache.flink.api.common.io.RichOutputFormat; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; -import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; - -/** - * OutputFormat to write tuples into a database. - * The OutputFormat has to be configured using the supplied OutputFormatBuilder. - * - * @see Tuple - * @see DriverManager - */ -public class RetractJDBCOutputFormat extends RichOutputFormat { - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); - - private String username; - private String password; - private String drivername; - private String dbURL; - private String insertQuery; - private String tableName; - private int batchInterval = 5000; - - private Connection dbConn; - private PreparedStatement upload; - - private int batchCount = 0; - - public int[] typesArray; - - private transient Counter outRecords; - - private transient Meter outRecordsRate; - - public RetractJDBCOutputFormat() { - } - - @Override - public void configure(Configuration parameters) { - } - - /** - * Connects to the target database and initializes the prepared statement. - * - * @param taskNumber The number of the parallel instance. - * @throws IOException Thrown, if the output could not be opened due to an - * I/O problem. - */ - @Override - public void open(int taskNumber, int numTasks) throws IOException { - try { - establishConnection(); - upload = dbConn.prepareStatement(insertQuery); - initMetric(); - if (dbConn.getMetaData().getTables(null, null, tableName, null).next()){ - upload = dbConn.prepareStatement(insertQuery); - } else { - throw new SQLException("Table " + tableName +" doesn't exist"); - } - - } catch (SQLException sqe) { - throw new IllegalArgumentException("open() failed.", sqe); - } catch (ClassNotFoundException cnfe) { - throw new IllegalArgumentException("JDBC driver class not found.", cnfe); - } - } - - private void initMetric(){ - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - - private void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(drivername); - if (username == null) { - dbConn = DriverManager.getConnection(dbURL); - } else { - dbConn = DriverManager.getConnection(dbURL, username, password); - } - } - - /** - * Adds a record to the prepared statement. - *

- * When this method is called, the output format is guaranteed to be opened. - *

- * - * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to - * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) - * - * @param tuple2 The records to add to the output. - * @see PreparedStatement - * @throws IOException Thrown, if the records could not be added due to an I/O problem. - */ - @Override - public void writeRecord(Tuple2 tuple2) throws IOException { - - Tuple2 tupleTrans = tuple2; - Boolean retract = tupleTrans.getField(0); - Row row = tupleTrans.getField(1); - - - if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { - LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); - } - try { - if(retract){ - insertWrite(row); - outRecords.inc(); - }else{ - //do nothing - } - } catch (SQLException | IllegalArgumentException e) { - throw new IllegalArgumentException("writeRecord() failed", e); - } - } - - - private void insertWrite(Row row) throws SQLException { - - updatePreparedStmt(row, upload); - upload.addBatch(); - batchCount++; - if (batchCount >= batchInterval) { - upload.executeBatch(); - batchCount = 0; - } - } - - - private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { - if (typesArray == null ) { - // no types provided - for (int index = 0; index < row.getArity(); index++) { - LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); - pstmt.setObject(index + 1, row.getField(index)); - } - } else { - // types provided - for (int index = 0; index < row.getArity(); index++) { - - if (row.getField(index) == null) { - pstmt.setNull(index + 1, typesArray[index]); - } else { - // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html - switch (typesArray[index]) { - case java.sql.Types.NULL: - pstmt.setNull(index + 1, typesArray[index]); - break; - case java.sql.Types.BOOLEAN: - case java.sql.Types.BIT: - pstmt.setBoolean(index + 1, (boolean) row.getField(index)); - break; - case java.sql.Types.CHAR: - case java.sql.Types.NCHAR: - case java.sql.Types.VARCHAR: - case java.sql.Types.LONGVARCHAR: - case java.sql.Types.LONGNVARCHAR: - pstmt.setString(index + 1, (String) row.getField(index)); - break; - case java.sql.Types.TINYINT: - pstmt.setByte(index + 1, (byte) row.getField(index)); - break; - case java.sql.Types.SMALLINT: - pstmt.setShort(index + 1, (short) row.getField(index)); - break; - case java.sql.Types.INTEGER: - pstmt.setInt(index + 1, (int) row.getField(index)); - break; - case java.sql.Types.BIGINT: - pstmt.setLong(index + 1, (long) row.getField(index)); - break; - case java.sql.Types.REAL: - case java.sql.Types.FLOAT: - pstmt.setFloat(index + 1, (float) row.getField(index)); - break; - case java.sql.Types.DOUBLE: - pstmt.setDouble(index + 1, (double) row.getField(index)); - break; - case java.sql.Types.DECIMAL: - case java.sql.Types.NUMERIC: - pstmt.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); - break; - case java.sql.Types.DATE: - pstmt.setDate(index + 1, (java.sql.Date) row.getField(index)); - break; - case java.sql.Types.TIME: - pstmt.setTime(index + 1, (java.sql.Time) row.getField(index)); - break; - case java.sql.Types.TIMESTAMP: - pstmt.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); - break; - case java.sql.Types.BINARY: - case java.sql.Types.VARBINARY: - case java.sql.Types.LONGVARBINARY: - pstmt.setBytes(index + 1, (byte[]) row.getField(index)); - break; - default: - pstmt.setObject(index + 1, row.getField(index)); - LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", - typesArray[index], index + 1, row.getField(index)); - // case java.sql.Types.SQLXML - // case java.sql.Types.ARRAY: - // case java.sql.Types.JAVA_OBJECT: - // case java.sql.Types.BLOB: - // case java.sql.Types.CLOB: - // case java.sql.Types.NCLOB: - // case java.sql.Types.DATALINK: - // case java.sql.Types.DISTINCT: - // case java.sql.Types.OTHER: - // case java.sql.Types.REF: - // case java.sql.Types.ROWID: - // case java.sql.Types.STRUC - } - } - } - } - } - - /** - * Executes prepared statement and closes all resources of this instance. - * - * @throws IOException Thrown, if the input could not be closed properly. - */ - @Override - public void close() throws IOException { - try { - if (upload != null) { - upload.executeBatch(); - upload.close(); - } - } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } finally { - upload = null; - batchCount = 0; - } - - try { - if (dbConn != null) { - dbConn.close(); - } - } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } finally { - dbConn = null; - } - } - - public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { - return new JDBCOutputFormatBuilder(); - } - - public static class JDBCOutputFormatBuilder { - private final RetractJDBCOutputFormat format; - - protected JDBCOutputFormatBuilder() { - this.format = new RetractJDBCOutputFormat(); - } - - public JDBCOutputFormatBuilder setUsername(String username) { - format.username = username; - return this; - } - - public JDBCOutputFormatBuilder setPassword(String password) { - format.password = password; - return this; - } - - public JDBCOutputFormatBuilder setDrivername(String drivername) { - format.drivername = drivername; - return this; - } - - public JDBCOutputFormatBuilder setDBUrl(String dbURL) { - format.dbURL = dbURL; - return this; - } - - public JDBCOutputFormatBuilder setInsertQuery(String query) { - format.insertQuery = query; - return this; - } - - - public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { - format.batchInterval = batchInterval; - return this; - } - - public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { - format.typesArray = typesArray; - return this; - } - - public JDBCOutputFormatBuilder setTableName(String tableName) { - format.tableName = tableName; - return this; - } - - /** - * Finalizes the configuration and checks validity. - * - * @return Configured RetractJDBCOutputFormat - */ - public RetractJDBCOutputFormat finish() { - if (format.username == null) { - LOG.info("Username was not supplied separately."); - } - if (format.password == null) { - LOG.info("Password was not supplied separately."); - } - if (format.dbURL == null) { - throw new IllegalArgumentException("No dababase URL supplied."); - } - if (format.insertQuery == null) { - throw new IllegalArgumentException("No insertQuery suplied"); - } - if (format.drivername == null) { - throw new IllegalArgumentException("No driver supplied"); - } - return format; - } - } - -} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java index f1522226b..2247eb8cc 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java @@ -16,13 +16,11 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.mysql.table; -import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -30,26 +28,17 @@ * Reason: * Date: 2018/7/4 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSinkParser extends AbsTableParser { +public class MysqlSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "mysql"; @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MysqlTableInfo mysqlTableInfo = new MysqlTableInfo(); - mysqlTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, mysqlTableInfo); - - mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlTableInfo.PARALLELISM_KEY.toLowerCase()))); - mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlTableInfo.URL_KEY.toLowerCase()))); - mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlTableInfo.TABLE_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlTableInfo.USER_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlTableInfo.PASSWORD_KEY.toLowerCase()))); - mysqlTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(MysqlTableInfo.BATCH_SIZE_KEY.toLowerCase()))); - mysqlTableInfo.setBufferSize(MathUtil.getString(props.get(MysqlTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); - mysqlTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(MysqlTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); - + TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } } diff --git a/mysql/pom.xml b/mysql/pom.xml index 747646f74..36221bcff 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -6,8 +6,10 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 + sql.mysql pom @@ -16,6 +18,11 @@ mysql-side + + 5.1.46 + 1.0-SNAPSHOT + + junit @@ -23,16 +30,18 @@ 3.8.1 test + com.dtstack.flink sql.core - 1.0-SNAPSHOT + ${sql.core.version} provided + mysql mysql-connector-java - 5.1.46 + ${mysql.connector.version} diff --git a/rdb/pom.xml b/rdb/pom.xml index 0ef3aa252..5cd5d4187 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -6,6 +6,7 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 15f1ee923..66945c513 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -58,10 +58,6 @@ public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List resultFuture) throws Exception { @@ -176,7 +172,10 @@ public Row fillData(Row input, Object line) { @Override public void close() throws Exception { super.close(); - rdbSQLClient.close(); + if (rdbSQLClient != null) { + rdbSQLClient.close(); + } + } public String buildCacheKey(JsonArray jsonArray) { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index 2117639c4..c08f9a3a9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.rdb.table; import com.dtstack.flink.sql.side.SideTableInfo; diff --git a/rdb/rdb-sink/pom.xml b/rdb/rdb-sink/pom.xml index 6b4868c95..176615769 100644 --- a/rdb/rdb-sink/pom.xml +++ b/rdb/rdb-sink/pom.xml @@ -6,6 +6,7 @@ sql.rdb com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java similarity index 79% rename from mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java index c3526ecd1..35ecf926c 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,19 +6,16 @@ * 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 - * + *

+ * 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 com.dtstack.flink.sql.sink.mysql; +package com.dtstack.flink.sql.sink.rdb; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -38,11 +35,12 @@ import java.util.List; /** - * Date: 2017/2/27 + * Reason: + * Date: 2018/11/27 * Company: www.dtstack.com - * @author xuchao + * + * @author maqi */ - public abstract class DBSink implements RetractStreamTableSink { protected String driverName; @@ -69,10 +67,10 @@ public abstract class DBSink implements RetractStreamTableSink { private int parallelism = -1; - public RichSinkFunction createJdbcSinkFunc(){ + public RichSinkFunction createJdbcSinkFunc() { - if(driverName == null || dbURL == null || userName == null - || password == null || sqlTypes == null || tableName == null){ + if (driverName == null || dbURL == null || userName == null + || password == null || sqlTypes == null || tableName == null) { throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + " must not be null. please check it!!!"); } @@ -95,34 +93,35 @@ public RichSinkFunction createJdbcSinkFunc(){ /** * By now specified class type conversion. * FIXME Follow-up has added a new type of time needs to be modified + * * @param fieldTypeArray */ - protected void buildSqlTypes(List fieldTypeArray){ + protected void buildSqlTypes(List fieldTypeArray) { int[] tmpFieldsType = new int[fieldTypeArray.size()]; - for(int i=0; i fieldTypeArray){ /** * Set the default frequency submit updated every submission + * * @param batchInterval */ public void setBatchInterval(int batchInterval) { @@ -143,7 +143,7 @@ public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createJdbcSinkFunc(); DataStreamSink streamSink = dataStream.addSink(richSinkFunction); streamSink.name(tableName); - if(parallelism > 0){ + if (parallelism > 0) { streamSink.setParallelism(parallelism); } } @@ -176,11 +176,15 @@ public TypeInformation[] getFieldTypes() { } - public void setParallelism(int parallelism){ + public void setParallelism(int parallelism) { this.parallelism = parallelism; } - public void buildSql(String tableName, List fields){ - throw new RuntimeException("you need to overwrite this method in your own class."); - } + /** + * you need to implements this method in your own class. + * + * @param tableName + * @param fields + */ + public abstract void buildSql(String tableName, List fields); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java new file mode 100644 index 000000000..ae1630a38 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -0,0 +1,71 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; + +import java.util.Arrays; +import java.util.List; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public abstract class RdbSink extends DBSink implements IStreamSinkGener { + + @Override + public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { + RdbTableInfo rdbTableInfo = (RdbTableInfo) targetTableInfo; + + String tmpDbURL = rdbTableInfo.getUrl(); + String tmpUserName = rdbTableInfo.getUserName(); + String tmpPassword = rdbTableInfo.getPassword(); + String tmpTableName = rdbTableInfo.getTableName(); + + Integer tmpSqlBatchSize = rdbTableInfo.getBatchSize(); + if (tmpSqlBatchSize != null) { + setBatchInterval(tmpSqlBatchSize); + } + + Integer tmpSinkParallelism = rdbTableInfo.getParallelism(); + if (tmpSinkParallelism != null) { + setParallelism(tmpSinkParallelism); + } + + List fields = Arrays.asList(rdbTableInfo.getFields()); + List fieldTypeArray = Arrays.asList(rdbTableInfo.getFieldClasses()); + + this.driverName = getDriverName(); + this.dbURL = tmpDbURL; + this.userName = tmpUserName; + this.password = tmpPassword; + this.tableName = tmpTableName; + this.primaryKeys = rdbTableInfo.getPrimaryKeys(); + buildSql(tableName, fields); + buildSqlTypes(fieldTypeArray); + return this; + } + + public abstract String getDriverName(); + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java new file mode 100644 index 000000000..a47b6472c --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -0,0 +1,362 @@ +/* + * 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 com.dtstack.flink.sql.sink.rdb; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; + +/** + * OutputFormat to write tuples into a database. + * The OutputFormat has to be configured using the supplied OutputFormatBuilder. + * + * @see Tuple + * @see DriverManager + */ +public class RetractJDBCOutputFormat extends RichOutputFormat { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); + + private String username; + private String password; + private String drivername; + private String dbURL; + private String insertQuery; + private String tableName; + private int batchInterval = 5000; + + private Connection dbConn; + private PreparedStatement upload; + + private int batchCount = 0; + + public int[] typesArray; + + private transient Counter outRecords; + + private transient Meter outRecordsRate; + + public RetractJDBCOutputFormat() { + } + + @Override + public void configure(Configuration parameters) { + } + + /** + * Connects to the target database and initializes the prepared statement. + * + * @param taskNumber The number of the parallel instance. + * @throws IOException Thrown, if the output could not be opened due to an + * I/O problem. + */ + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + initMetric(); + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + upload = dbConn.prepareStatement(insertQuery); + } else { + throw new SQLException("Table " + tableName + " doesn't exist"); + } + + } catch (SQLException sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } catch (ClassNotFoundException cnfe) { + throw new IllegalArgumentException("JDBC driver class not found.", cnfe); + } + } + + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + + private void establishConnection() throws SQLException, ClassNotFoundException { + Class.forName(drivername); + if (username == null) { + dbConn = DriverManager.getConnection(dbURL); + } else { + dbConn = DriverManager.getConnection(dbURL, username, password); + } + } + + /** + * Adds a record to the prepared statement. + *

+ * When this method is called, the output format is guaranteed to be opened. + *

+ *

+ * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to + * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) + * + * @param tuple2 The records to add to the output. + * @throws IOException Thrown, if the records could not be added due to an I/O problem. + * @see PreparedStatement + */ + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + Row row = tupleTrans.getField(1); + + + if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { + LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); + } + try { + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing + } + } catch (SQLException | IllegalArgumentException e) { + throw new IllegalArgumentException("writeRecord() failed", e); + } + } + + + private void insertWrite(Row row) throws SQLException { + + updatePreparedStmt(row, upload); + upload.addBatch(); + batchCount++; + if (batchCount >= batchInterval) { + upload.executeBatch(); + batchCount = 0; + } + } + + + private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { + if (typesArray == null) { + // no types provided + for (int index = 0; index < row.getArity(); index++) { + LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); + pstmt.setObject(index + 1, row.getField(index)); + } + } else { + // types provided + for (int index = 0; index < row.getArity(); index++) { + + if (row.getField(index) == null) { + pstmt.setNull(index + 1, typesArray[index]); + } else { + // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html + switch (typesArray[index]) { + case java.sql.Types.NULL: + pstmt.setNull(index + 1, typesArray[index]); + break; + case java.sql.Types.BOOLEAN: + case java.sql.Types.BIT: + pstmt.setBoolean(index + 1, (boolean) row.getField(index)); + break; + case java.sql.Types.CHAR: + case java.sql.Types.NCHAR: + case java.sql.Types.VARCHAR: + case java.sql.Types.LONGVARCHAR: + case java.sql.Types.LONGNVARCHAR: + pstmt.setString(index + 1, (String) row.getField(index)); + break; + case java.sql.Types.TINYINT: + pstmt.setByte(index + 1, (byte) row.getField(index)); + break; + case java.sql.Types.SMALLINT: + pstmt.setShort(index + 1, (short) row.getField(index)); + break; + case java.sql.Types.INTEGER: + pstmt.setInt(index + 1, (int) row.getField(index)); + break; + case java.sql.Types.BIGINT: + pstmt.setLong(index + 1, (long) row.getField(index)); + break; + case java.sql.Types.REAL: + case java.sql.Types.FLOAT: + pstmt.setFloat(index + 1, (float) row.getField(index)); + break; + case java.sql.Types.DOUBLE: + pstmt.setDouble(index + 1, (double) row.getField(index)); + break; + case java.sql.Types.DECIMAL: + case java.sql.Types.NUMERIC: + pstmt.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); + break; + case java.sql.Types.DATE: + pstmt.setDate(index + 1, (java.sql.Date) row.getField(index)); + break; + case java.sql.Types.TIME: + pstmt.setTime(index + 1, (java.sql.Time) row.getField(index)); + break; + case java.sql.Types.TIMESTAMP: + pstmt.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); + break; + case java.sql.Types.BINARY: + case java.sql.Types.VARBINARY: + case java.sql.Types.LONGVARBINARY: + pstmt.setBytes(index + 1, (byte[]) row.getField(index)); + break; + default: + pstmt.setObject(index + 1, row.getField(index)); + LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", + typesArray[index], index + 1, row.getField(index)); + // case java.sql.Types.SQLXML + // case java.sql.Types.ARRAY: + // case java.sql.Types.JAVA_OBJECT: + // case java.sql.Types.BLOB: + // case java.sql.Types.CLOB: + // case java.sql.Types.NCLOB: + // case java.sql.Types.DATALINK: + // case java.sql.Types.DISTINCT: + // case java.sql.Types.OTHER: + // case java.sql.Types.REF: + // case java.sql.Types.ROWID: + // case java.sql.Types.STRUC + } + } + } + } + } + + /** + * Executes prepared statement and closes all resources of this instance. + * + * @throws IOException Thrown, if the input could not be closed properly. + */ + @Override + public void close() throws IOException { + try { + if (upload != null) { + upload.executeBatch(); + upload.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + upload = null; + batchCount = 0; + } + + try { + if (dbConn != null) { + dbConn.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + dbConn = null; + } + } + + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { + return new JDBCOutputFormatBuilder(); + } + + public static class JDBCOutputFormatBuilder { + private final RetractJDBCOutputFormat format; + + protected JDBCOutputFormatBuilder() { + this.format = new RetractJDBCOutputFormat(); + } + + public JDBCOutputFormatBuilder setUsername(String username) { + format.username = username; + return this; + } + + public JDBCOutputFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public JDBCOutputFormatBuilder setDrivername(String drivername) { + format.drivername = drivername; + return this; + } + + public JDBCOutputFormatBuilder setDBUrl(String dbURL) { + format.dbURL = dbURL; + return this; + } + + public JDBCOutputFormatBuilder setInsertQuery(String query) { + format.insertQuery = query; + return this; + } + + + public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { + format.batchInterval = batchInterval; + return this; + } + + public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { + format.typesArray = typesArray; + return this; + } + + public JDBCOutputFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public RetractJDBCOutputFormat finish() { + if (format.username == null) { + LOG.info("Username was not supplied separately."); + } + if (format.password == null) { + LOG.info("Password was not supplied separately."); + } + if (format.dbURL == null) { + throw new IllegalArgumentException("No dababase URL supplied."); + } + if (format.insertQuery == null) { + throw new IllegalArgumentException("No insertQuery suplied"); + } + if (format.drivername == null) { + throw new IllegalArgumentException("No driver supplied"); + } + return format; + } + } + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java new file mode 100644 index 000000000..a14825a67 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -0,0 +1,51 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class RdbSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RdbTableInfo rdbTableInfo = new RdbTableInfo(); + rdbTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, rdbTableInfo); + + rdbTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(RdbTableInfo.PARALLELISM_KEY.toLowerCase()))); + rdbTableInfo.setUrl(MathUtil.getString(props.get(RdbTableInfo.URL_KEY.toLowerCase()))); + rdbTableInfo.setTableName(MathUtil.getString(props.get(RdbTableInfo.TABLE_NAME_KEY.toLowerCase()))); + rdbTableInfo.setUserName(MathUtil.getString(props.get(RdbTableInfo.USER_NAME_KEY.toLowerCase()))); + rdbTableInfo.setPassword(MathUtil.getString(props.get(RdbTableInfo.PASSWORD_KEY.toLowerCase()))); + rdbTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(RdbTableInfo.BATCH_SIZE_KEY.toLowerCase()))); + rdbTableInfo.setBufferSize(MathUtil.getString(props.get(RdbTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); + rdbTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(RdbTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); + + return rdbTableInfo; + } +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java similarity index 81% rename from mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index 0576c466e..f170b88e1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,33 +6,30 @@ * 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 - * + *

+ * 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 com.dtstack.flink.sql.sink.mysql.table; +package com.dtstack.flink.sql.sink.rdb.table; import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; /** * Reason: - * Date: 2018/6/22 + * Date: 2018/11/27 * Company: www.dtstack.com - * @author xuchao + * + * @author maqi */ +public class RdbTableInfo extends TargetTableInfo { -public class MysqlTableInfo extends TargetTableInfo { - - private static final String CURR_TYPE = "mysql"; + //private static final String CURR_TYPE = "mysql"; public static final String URL_KEY = "url"; @@ -48,10 +45,6 @@ public class MysqlTableInfo extends TargetTableInfo { public static final String FLUSH_INTERVALMS_KEY = "flushIntervalMs"; - public MysqlTableInfo(){ - setType(CURR_TYPE); - } - private String url; private String tableName; @@ -124,10 +117,10 @@ public void setFlushIntervalMs(String flushIntervalMs) { @Override public boolean check() { - Preconditions.checkNotNull(url, "mysql field of URL is required"); - Preconditions.checkNotNull(tableName, "mysql field of tableName is required"); - Preconditions.checkNotNull(userName, "mysql field of userName is required"); - Preconditions.checkNotNull(password, "mysql field of password is required"); + Preconditions.checkNotNull(url, "rdb field of URL is required"); + Preconditions.checkNotNull(tableName, "rdb field of tableName is required"); + Preconditions.checkNotNull(userName, "rdb field of userName is required"); + Preconditions.checkNotNull(password, "rdb field of password is required"); return true; } diff --git a/sqlserver/pom.xml b/sqlserver/pom.xml index 463fd55c8..de00840e8 100644 --- a/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -6,30 +6,25 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.sqlserver 1.0-SNAPSHOT - - sqlserver-side - pom 1.3.1 1.0-SNAPSHOT - 1.0-SNAPSHOT - - - com.dtstack.flink - sql.side.rdb - ${sql.side.rdb.version} - provided - + + sqlserver-side + sqlserver-sink + + com.dtstack.flink sql.core diff --git a/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml index dac53bed0..3848021e6 100644 --- a/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -6,12 +6,12 @@ sql.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.side.sqlserver 1.0-SNAPSHOT - sqlserver-side pom @@ -32,8 +32,6 @@ sql.side.rdb ${rdb.side.version} - - diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index b188aaedf..7e3db7ecd 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -6,6 +6,7 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java index ef2786f57..d123c8f9c 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver; diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java index f671dc0e6..2b9913e7e 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver; diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml index 08413dc43..902228d01 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml @@ -6,6 +6,7 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 1774d4388..4f39129ca 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -25,7 +25,6 @@ import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; import io.vertx.core.json.JsonObject; -import io.vertx.ext.sql.SQLClient; import io.vertx.ext.jdbc.JDBCClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index aed72e25f..dcff068b1 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -6,13 +6,14 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.side.sqlserver.core 1.0-SNAPSHOT jar - + sqlserver-side-core \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java index edd4820be..3631dcd8a 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; diff --git a/sqlserver/sqlserver-sink/pom.xml b/sqlserver/sqlserver-sink/pom.xml new file mode 100644 index 000000000..1180f1ca8 --- /dev/null +++ b/sqlserver/sqlserver-sink/pom.xml @@ -0,0 +1,92 @@ + + + + sql.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.sqlserver + 1.0-SNAPSHOT + sqlserver-sink + + + 1.0-SNAPSHOT + + + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java new file mode 100644 index 000000000..986486445 --- /dev/null +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -0,0 +1,47 @@ +/** + * 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 com.dtstack.flink.sql.sink.sqlserver; + +import com.dtstack.flink.sql.sink.rdb.RdbSink; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverSink extends RdbSink { + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + @Override + public String getDriverName() { + return SQLSERVER_DRIVER; + } + + @Override + public void buildSql(String tableName, List fields) { + buildInsertSql(tableName, fields); + } + + private void buildInsertSql(String tableName, List fields) { + + } +} diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java new file mode 100644 index 000000000..a695d6c3b --- /dev/null +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java @@ -0,0 +1,41 @@ +/** + * 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 com.dtstack.flink.sql.sink.sqlserver.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "sqlserver"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlserverTableInfo.setType(CURR_TYPE); + return sqlserverTableInfo; + } +} From e6035bf03dda240171d32aff7638214deb0c57f8 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 27 Nov 2018 20:45:30 +0800 Subject: [PATCH 29/42] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 27dffc8dc..3abfabbc3 100644 --- a/README.md +++ b/README.md @@ -17,6 +17,7 @@ * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 + * sql优化(谓词下移等) ## 1 快速起步 ### 1.1 运行模式 From f9b6efdc4e17f8852543a075c798c55ac7d47244 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 27 Nov 2018 21:16:12 +0800 Subject: [PATCH 30/42] fetch size --- .../dtstack/flink/sql/side/mysql/MysqlAllReqRow.java | 5 +++++ .../dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 11 ++++++----- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 3 +-- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index fa4895ded..b56c3252b 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -67,4 +67,9 @@ public Connection getConn(String dbURL, String userName, String password) { throw new RuntimeException("", e); } } + + @Override + public int getFetchSize(){ + return Integer.MIN_VALUE; + } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 118ee048f..54ea47a18 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -54,12 +54,8 @@ public abstract class RdbAllReqRow extends AllReqRow { private static final Logger LOG = LoggerFactory.getLogger(RdbAllReqRow.class); - //private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; - private static final int FETCH_SIZE = 1000; - private AtomicReference>>> cacheRef = new AtomicReference<>(); public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { @@ -195,7 +191,7 @@ private void loadData(Map>> tmpCache) throws SQ //load data from table String sql = sideInfo.getSqlCondition(); Statement statement = connection.createStatement(); - statement.setFetchSize(FETCH_SIZE); + statement.setFetchSize(getFetchSize()); ResultSet resultSet = statement.executeQuery(sql); String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); while (resultSet.next()) { @@ -216,4 +212,9 @@ private void loadData(Map>> tmpCache) throws SQ } } } + + public int getFetchSize(){ + return 1000; + } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 66945c513..61aba3e40 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -105,8 +105,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except String sqlCondition = sideInfo.getSqlCondition(); connection.queryWithParams(sqlCondition, inputParams, rs -> { if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); + LOG.error("Cannot retrieve the data from the database", rs.cause()); resultFuture.complete(null); return; } From 2c2ae4dbca2057fa661ec9d50c097d53d12ed384 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 27 Nov 2018 21:19:25 +0800 Subject: [PATCH 31/42] add metric --- .../flink/sql/sink/hbase/HbaseOutputFormat.java | 15 +++++++++++++++ .../flink/sql/sink/mongo/MongoOutputFormat.java | 15 +++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index cc95d1e75..ea47b5689 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,10 +20,14 @@ package com.dtstack.flink.sql.sink.hbase; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -62,6 +66,10 @@ public class HbaseOutputFormat extends RichOutputFormat { private transient Connection conn; private transient Table table; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @@ -82,6 +90,7 @@ public void open(int taskNumber, int numTasks) throws IOException { conn = ConnectionFactory.createConnection(conf); table = conn.getTable(TableName.valueOf(tableName)); LOG.warn("---open end(get table from hbase) ---"); + initMetric(); } @Override @@ -133,7 +142,13 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } table.put(put); + outRecords.inc(); + + } + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); } @Override diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index f86be6212..b32177aab 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; +import com.dtstack.flink.sql.metric.MetricConstant; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; @@ -31,6 +32,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.bson.Document; import org.bson.types.ObjectId; @@ -64,6 +68,10 @@ public class MongoOutputFormat extends RichOutputFormat { private static String PK = "_ID"; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); @Override @@ -73,6 +81,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); + initMetric(); } @Override @@ -107,6 +116,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } else { dbCollection.insertOne(doc); } + outRecords.inc(); } @Override @@ -147,6 +157,11 @@ private void establishConnection() { } } + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + private MongoOutputFormat() { } From 1bf0b00e98c619c037c57bea13d831f6fd283d72 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:36:23 +0800 Subject: [PATCH 32/42] abstract output metric --- .../flink/sql/metric/MetricOutputFormat.java | 24 +++++++++++++++++++ .../sql/sink/hbase/HbaseOutputFormat.java | 18 ++------------ .../sql/sink/mongo/MongoOutputFormat.java | 18 ++------------ .../sql/sink/rdb/RetractJDBCOutputFormat.java | 19 +++------------ .../sql/sink/redis/RedisOutputFormat.java | 18 ++------------ 5 files changed, 33 insertions(+), 64 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java new file mode 100644 index 000000000..e6b3d88b0 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java @@ -0,0 +1,24 @@ +package com.dtstack.flink.sql.metric; + +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; + +/** + * Created by sishu.yss on 2018/11/28. + */ +public abstract class MetricOutputFormat extends RichOutputFormat{ + + protected transient Counter outRecords; + + protected transient Meter outRecordsRate; + + public void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index ea47b5689..5dbeea040 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,14 +20,10 @@ package com.dtstack.flink.sql.sink.hbase; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -38,7 +34,6 @@ import org.apache.hadoop.hbase.client.Table; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -48,7 +43,7 @@ * author: jingzhen@dtstack.com * date: 2017-6-29 */ -public class HbaseOutputFormat extends RichOutputFormat { +public class HbaseOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); @@ -66,10 +61,6 @@ public class HbaseOutputFormat extends RichOutputFormat { private transient Connection conn; private transient Table table; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @@ -146,11 +137,6 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - @Override public void close() throws IOException { if(conn != null) { diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index b32177aab..10651871c 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; @@ -28,19 +28,14 @@ import com.mongodb.client.MongoDatabase; import com.mongodb.client.result.UpdateResult; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.bson.Document; import org.bson.types.ObjectId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -52,7 +47,7 @@ * * @author xuqianjin */ -public class MongoOutputFormat extends RichOutputFormat { +public class MongoOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(MongoOutputFormat.class); private String address; @@ -68,10 +63,6 @@ public class MongoOutputFormat extends RichOutputFormat { private static String PK = "_ID"; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); @Override @@ -157,11 +148,6 @@ private void establishConnection() { } } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private MongoOutputFormat() { } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index a47b6472c..2635268ff 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -18,23 +18,18 @@ package com.dtstack.flink.sql.sink.rdb; -import com.dtstack.flink.sql.metric.MetricConstant; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; +import com.dtstack.flink.sql.metric.MetricOutputFormat; /** * OutputFormat to write tuples into a database. @@ -43,7 +38,7 @@ * @see Tuple * @see DriverManager */ -public class RetractJDBCOutputFormat extends RichOutputFormat { +public class RetractJDBCOutputFormat extends MetricOutputFormat { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); @@ -63,11 +58,8 @@ public class RetractJDBCOutputFormat extends RichOutputFormat { public int[] typesArray; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public RetractJDBCOutputFormat() { + } @Override @@ -100,11 +92,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); if (username == null) { diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index f43114918..ca305f046 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,23 +18,18 @@ package com.dtstack.flink.sql.sink.redis; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import redis.clients.jedis.*; - import java.io.Closeable; import java.io.IOException; import java.util.*; -public class RedisOutputFormat extends RichOutputFormat { +public class RedisOutputFormat extends MetricOutputFormat { private String url; @@ -70,10 +65,6 @@ public class RedisOutputFormat extends RichOutputFormat { private GenericObjectPoolConfig poolConfig; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - private RedisOutputFormat(){ } @Override @@ -101,11 +92,6 @@ private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, S return config; } - private void initMetric(){ - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private void establishConnection() { poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); From 197d52149abbdb613b6a47ac1e5b338c444d8ca2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:37:00 +0800 Subject: [PATCH 33/42] commnet --- .../flink/sql/metric/MetricOutputFormat.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java index e6b3d88b0..bbf1152a3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java @@ -1,3 +1,20 @@ +/* + * 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 com.dtstack.flink.sql.metric; import org.apache.flink.api.java.tuple.Tuple2; From c1f29d2457b7513ea0667d184900251492b99239 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 28 Nov 2018 09:49:47 +0800 Subject: [PATCH 34/42] Update README.md --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 3abfabbc3..b6a39720c 100644 --- a/README.md +++ b/README.md @@ -5,6 +5,8 @@ > > * 自定义create function 语法 > > * 实现了流与维表的join > > * 支持原生FLinkSQL所有的语法 +> > * 扩展了输入和输出的性能指标到promethus + # 已支持 * 源表:kafka 0.9,1.x版本 From 2a533b80c800c99fe0f4d3b5608a74434c4013b2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:58:57 +0800 Subject: [PATCH 35/42] rename package --- .../dtstack/flink/sql/{metric => sink}/MetricOutputFormat.java | 3 ++- .../com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java | 2 +- .../com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java | 2 +- .../dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java | 2 +- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{metric => sink}/MetricOutputFormat.java (94%) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java similarity index 94% rename from core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java rename to core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java index bbf1152a3..4e11280e7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flink.sql.metric; +package com.dtstack.flink.sql.sink; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.common.io.RichOutputFormat; diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index 5dbeea040..2ef2c6d52 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.hbase; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 10651871c..87ffc7354 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 2635268ff..21c2a537d 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -29,7 +29,7 @@ import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; /** * OutputFormat to write tuples into a database. diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index ca305f046..910882c9f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.redis; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; From 235d7429182ece2108be3c77aa18e8e9c790749e Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 28 Nov 2018 16:58:02 +0800 Subject: [PATCH 36/42] fix bugs --- .../sql/side/redis/table/RedisSideParser.java | 20 +++++++++---------- .../side/redis/table/RedisSideTableInfo.java | 8 ++++---- .../sql/sink/redis/table/RedisSinkParser.java | 10 +++++----- .../sql/sink/redis/table/RedisTableInfo.java | 4 ++-- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index fdeedfdd1..f8a83b868 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -33,20 +33,20 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 28 Nov 2018 22:19:03 +0800 Subject: [PATCH 37/42] add netty --- redis5/redis5-side/redis-async-side/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index 1aaacd4dc..e0e573904 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -23,6 +23,11 @@ lettuce-core 5.0.5.RELEASE + + io.netty + netty-all + 4.1.24.Final + From f696bfbf507975ad59a8514af13b27e2d06990c6 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 10:29:22 +0800 Subject: [PATCH 38/42] rename jarname --- elasticsearch5/elasticsearch5-sink/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index 68f464e48..a6453f6ce 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -10,7 +10,7 @@ 4.0.0 sql.sink.elasticsearch5 - elasticsearch5-sink + elasticsearch-sink @@ -48,7 +48,6 @@ org.apache.logging.log4j:log4j-to-slf4j - org.slf4j:slf4j-api From 56f70aec7fd4c646cac2ef845e034e6bdbfe240a Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 14:59:59 +0800 Subject: [PATCH 39/42] add elasticsearch metric --- .../sink/elasticsearch/CustomerSinkFunc.java | 8 +++ .../sink/elasticsearch/ElasticsearchSink.java | 3 +- .../MetricElasticsearchSink.java | 58 +++++++++++++++++++ 3 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java index 7ad2720b4..a95d477de 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java @@ -23,6 +23,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.types.Row; @@ -56,6 +57,8 @@ public class CustomerSinkFunc implements ElasticsearchSinkFunction { private List fieldTypes; + public transient Counter outRecords; + /** 默认分隔符为'_' */ private char sp = '_'; @@ -79,11 +82,16 @@ public void process(Tuple2 tuple2, RuntimeContext ctx, RequestIndexer indexer) { indexer.add(createIndexRequest(element)); + outRecords.inc(); }catch (Throwable e){ logger.error("", e); } } + public void setOutRecords(Counter outRecords) { + this.outRecords = outRecords; + } + private IndexRequest createIndexRequest(Row element) { List idFieldList = new ArrayList<>(); diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java index 1429d1d90..f372e4d1e 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java @@ -27,6 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -131,7 +132,7 @@ private RichSinkFunction createEsSinkFunction(){ CustomerSinkFunc customerSinkFunc = new CustomerSinkFunc(index, type, Arrays.asList(fieldNames), Arrays.asList(columnTypes), idIndexList); - return new org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink(userConfig, transports, customerSinkFunc); + return new MetricElasticsearchSink(userConfig, transports, customerSinkFunc); } @Override diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java new file mode 100644 index 000000000..709150848 --- /dev/null +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flink.sql.sink.elasticsearch; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; + +import java.util.List; +import java.util.Map; + +/** + * @Auther: jiangjunjie + * @Date: 2018/11/29 14:15 + * @Description: + */ +public class MetricElasticsearchSink extends org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink { + + protected CustomerSinkFunc customerSinkFunc; + + protected transient Meter outRecordsRate; + + public MetricElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { + super(userConfig, transportAddresses, elasticsearchSinkFunction); + this.customerSinkFunc = (CustomerSinkFunc) elasticsearchSinkFunction; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + initMetric(); + } + + public void initMetric() { + Counter counter = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + customerSinkFunc.setOutRecords(counter); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + } +} From 3832dca86bda1c3c393797d3472b7d38ae8f0212 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Thu, 29 Nov 2018 20:25:08 +0800 Subject: [PATCH 40/42] add sqlserver result table --- .../flink/sql/enums/EDatabaseType.java | 33 +++++ .../flink/sql/side/mysql/MysqlAllReqRow.java | 2 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 7 - .../flink/sql/sink/mysql/MysqlSink.java | 7 + .../sql/side/rdb/async/RdbAsyncReqRow.java | 7 +- .../dtstack/flink/sql/sink/rdb/DBSink.java | 25 +++- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 2 + .../sql/sink/rdb/RetractJDBCOutputFormat.java | 122 +++++++++++++++++- .../side/sqlserver/SqlserverAsyncReqRow.java | 6 - .../sql/sink/sqlserver/SqlserverSink.java | 117 ++++++++++++++++- 10 files changed, 305 insertions(+), 23 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java new file mode 100644 index 000000000..9b01bf052 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java @@ -0,0 +1,33 @@ +/* + * 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 com.dtstack.flink.sql.enums; + +/** + * Database type + * + * Company: www.dtstack.com + * @author jiangbo + */ +public enum EDatabaseType { + + MYSQL, + SQLSERVER, + ORACLE, + +} diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index b56c3252b..aca7593f7 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -69,7 +69,7 @@ public Connection getConn(String dbURL, String userName, String password) { } @Override - public int getFetchSize(){ + public int getFetchSize() { return Integer.MIN_VALUE; } } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 86fe8c8fa..456451214 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -49,13 +49,6 @@ public class MysqlAsyncReqRow extends RdbAsyncReqRow { private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; - - private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; - - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - - public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index fe190f10f..b4ff9dea1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.sink.rdb.RdbSink; import java.util.List; +import java.util.Map; /** * Date: 2017/2/27 @@ -43,6 +44,11 @@ public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); } + @Override + public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + return null; + } + private void buildInsertSql(String tableName, List fields) { String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; String fieldsStr = ""; @@ -66,4 +72,5 @@ public String getDriverName() { return MYSQL_DRIVER; } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 61aba3e40..a7e5339e9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -52,13 +52,18 @@ public class RdbAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); + public final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 1; + + public final static int DEFAULT_VERTX_WORKER_POOL_SIZE = Runtime.getRuntime().availableProcessors() * 2; + + public final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE + DEFAULT_VERTX_WORKER_POOL_SIZE; + private transient SQLClient rdbSQLClient; public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RdbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - @Override public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java index 35ecf926c..4f6372d54 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java @@ -21,6 +21,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; @@ -29,10 +30,12 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.io.Serializable; import java.math.BigDecimal; import java.sql.Timestamp; import java.sql.Types; import java.util.List; +import java.util.Map; /** * Reason: @@ -41,7 +44,7 @@ * * @author maqi */ -public abstract class DBSink implements RetractStreamTableSink { +public abstract class DBSink implements RetractStreamTableSink, Serializable { protected String driverName; @@ -51,6 +54,8 @@ public abstract class DBSink implements RetractStreamTableSink { protected String password; + protected String dbType; + protected int batchInterval = 1; protected int[] sqlTypes; @@ -67,6 +72,7 @@ public abstract class DBSink implements RetractStreamTableSink { private int parallelism = -1; + public RichSinkFunction createJdbcSinkFunc() { if (driverName == null || dbURL == null || userName == null @@ -84,6 +90,9 @@ public RichSinkFunction createJdbcSinkFunc() { jdbcFormatBuild.setBatchInterval(batchInterval); jdbcFormatBuild.setSqlTypes(sqlTypes); jdbcFormatBuild.setTableName(tableName); + jdbcFormatBuild.setDBType(dbType); + jdbcFormatBuild.setDBSink(this); + RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); @@ -180,6 +189,10 @@ public void setParallelism(int parallelism) { this.parallelism = parallelism; } + public void setDbType(String dbType) { + this.dbType = dbType; + } + /** * you need to implements this method in your own class. * @@ -187,4 +200,14 @@ public void setParallelism(int parallelism) { * @param fields */ public abstract void buildSql(String tableName, List fields); + + /** + * sqlserver and oracle maybe implement + * + * @param tableName + * @param fieldNames + * @param realIndexes + * @return + */ + public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index ae1630a38..586d4fb85 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -61,6 +61,8 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.password = tmpPassword; this.tableName = tmpTableName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); + this.dbType=rdbTableInfo.getType(); + buildSql(tableName, fields); buildSqlTypes(fieldTypeArray); return this; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 21c2a537d..ee1d03abc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -18,17 +18,20 @@ package com.dtstack.flink.sql.sink.rdb; +import com.dtstack.flink.sql.enums.EDatabaseType; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.sql.*; +import java.util.*; import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; + import com.dtstack.flink.sql.sink.MetricOutputFormat; /** @@ -49,15 +52,33 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String dbURL; private String insertQuery; private String tableName; + private String dbType; private int batchInterval = 5000; private Connection dbConn; private PreparedStatement upload; + //index field + private Map> realIndexes = Maps.newHashMap(); + //full field + private List fullField = Lists.newArrayList(); + + private DBSink dbSink; private int batchCount = 0; public int[] typesArray; + private final static String GET_ORACLE_INDEX_SQL = "SELECT " + + "t.INDEX_NAME," + + "t.COLUMN_NAME " + + "FROM " + + "user_ind_columns t," + + "user_indexes i " + + "WHERE " + + "t.index_name = i.index_name " + + "AND i.uniqueness = 'UNIQUE' " + + "AND t.table_name = '%s'"; + public RetractJDBCOutputFormat() { } @@ -77,9 +98,11 @@ public void configure(Configuration parameters) { public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); - upload = dbConn.prepareStatement(insertQuery); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + if (!EDatabaseType.MYSQL.name().equalsIgnoreCase(dbType) && isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.fieldNames), realIndexes, fullField); + } upload = dbConn.prepareStatement(insertQuery); } else { throw new SQLException("Table " + tableName + " doesn't exist"); @@ -92,6 +115,22 @@ public void open(int taskNumber, int numTasks) throws IOException { } } + private boolean isReplaceInsertQuery() throws SQLException { + getRealIndexes(); + getFullColumns(); + + if (!realIndexes.isEmpty()) { + for (List value : realIndexes.values()) { + for (String fieldName : dbSink.getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); if (username == null) { @@ -149,7 +188,6 @@ private void insertWrite(Row row) throws SQLException { } } - private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { if (typesArray == null) { // no types provided @@ -269,6 +307,67 @@ public void close() throws IOException { } } + /** + * get db all index + * + * @throws SQLException + */ + public void getRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs; + if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { + PreparedStatement ps = dbConn.prepareStatement(String.format(GET_ORACLE_INDEX_SQL, tableName)); + rs = ps.executeQuery(); + } else { + rs = dbConn.getMetaData().getIndexInfo(null, null, tableName, true, false); + } + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + realIndexes.put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void getFullColumns() throws SQLException { + String schema = null; + if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { + String[] parts = tableName.split("\\."); + if (parts.length == 2) { + schema = parts[0].toUpperCase(); + tableName = parts[1]; + } + } + + ResultSet rs = dbConn.getMetaData().getColumns(null, schema, tableName, null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + fullField.add(columnName.toUpperCase()); + } + } + + } + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { return new JDBCOutputFormatBuilder(); } @@ -321,6 +420,17 @@ public JDBCOutputFormatBuilder setTableName(String tableName) { return this; } + public JDBCOutputFormatBuilder setDBSink(DBSink dbSink) { + format.dbSink = dbSink; + return this; + } + + public JDBCOutputFormatBuilder setDBType(String dbType) { + format.dbType = dbType; + return this; + } + + /** * Finalizes the configuration and checks validity. * diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 4f39129ca..003d61b27 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -40,12 +40,6 @@ public class SqlserverAsyncReqRow extends RdbAsyncReqRow { private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; - - private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; - - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 986486445..164413216 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -18,8 +18,9 @@ package com.dtstack.flink.sql.sink.sqlserver; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import org.apache.commons.lang3.StringUtils; -import java.util.List; +import java.util.*; /** * Reason: @@ -42,6 +43,120 @@ public void buildSql(String tableName, List fields) { } private void buildInsertSql(String tableName, List fields) { + String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; + String fieldsStr = StringUtils.join(fields, ","); + String placeholder = ""; + for (String fieldName : fields) { + placeholder += ",?"; + } + placeholder = placeholder.replaceFirst(",", ""); + sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); + this.sql = sqlTmp; } + + @Override + public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + return "MERGE INTO " + tableName + " T1 USING " + + "(" + makeValues(fieldNames) + ") T2 ON (" + + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " + + getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)) + " WHEN NOT MATCHED THEN " + + "INSERT (" + quoteColumns(fieldNames) + ") VALUES (" + + quoteColumns(fieldNames, "T2") + ");"; + } + + + public String quoteColumns(List column) { + return quoteColumns(column, null); + } + + public String quoteColumns(List column, String table) { + String prefix = StringUtils.isBlank(table) ? "" : quoteTable(table) + "."; + List list = new ArrayList<>(); + for (String col : column) { + list.add(prefix + quoteColumn(col)); + } + return StringUtils.join(list, ","); + } + + protected List keyColList(Map> updateKey) { + List keyCols = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List list = entry.getValue(); + for (String col : list) { + if (!keyCols.contains(col)) { + keyCols.add(col); + } + } + } + return keyCols; + } + + public String getUpdateSql(List column, List fullColumn, String leftTable, String rightTable, List keyCols) { + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : quoteTable(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; + List list = new ArrayList<>(); + for (String col : fullColumn) { + if (keyCols == null || keyCols.size() == 0 ) { + continue; + } + if (fullColumn == null || column.contains(col)) { + list.add(prefixLeft + col + "=" + prefixRight + col); + } else { + list.add(prefixLeft + col + "=null"); + } + } + return StringUtils.join(list, ","); + } + + public String quoteTable(String table) { + String[] parts = table.split("\\."); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < parts.length; ++i) { + if (i != 0) { + sb.append("."); + } + sb.append(getStartQuote() + parts[i] + getEndQuote()); + } + return sb.toString(); + } + + + public String updateKeySql(Map> updateKey) { + List exprList = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List colList = new ArrayList<>(); + for (String col : entry.getValue()) { + colList.add("T1." + quoteColumn(col) + "=T2." + quoteColumn(col)); + } + exprList.add(StringUtils.join(colList, " AND ")); + } + return StringUtils.join(exprList, " OR "); + } + + + public String makeValues(List column) { + StringBuilder sb = new StringBuilder("SELECT "); + for (int i = 0; i < column.size(); ++i) { + if (i != 0) { + sb.append(","); + } + sb.append("? " + quoteColumn(column.get(i))); + } + return sb.toString(); + } + + public String quoteColumn(String column) { + return getStartQuote() + column + getEndQuote(); + } + + public String getStartQuote() { + return "\""; + } + + public String getEndQuote() { + return "\""; + } + + } From 25cc3451e6f40c3fd47a04c33a7fa968fd163c95 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 17:57:21 +0800 Subject: [PATCH 41/42] extract outputformat --- .../flink/sql/sink/mysql/MysqlSink.java | 3 +- .../dtstack/flink/sql/sink/rdb/DBSink.java | 213 ---------------- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 199 ++++++++++++++- .../sink/rdb/format/OracleOutputFormat.java | 115 +++++++++ .../{ => format}/RetractJDBCOutputFormat.java | 236 ++++++------------ .../rdb/format/SqlserverOutputFormat.java | 102 ++++++++ .../sql/sink/sqlserver/SqlserverSink.java | 5 +- 7 files changed, 492 insertions(+), 381 deletions(-) delete mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java (66%) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index b4ff9dea1..007c9a262 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.sink.mysql; +import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; import java.util.List; @@ -32,7 +33,7 @@ * @author xuchao */ -public class MysqlSink extends RdbSink { +public class MysqlSink extends RdbSink implements IStreamSinkGener { private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java deleted file mode 100644 index 4f6372d54..000000000 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java +++ /dev/null @@ -1,213 +0,0 @@ -/** - * 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 com.dtstack.flink.sql.sink.rdb; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; - -import java.io.Serializable; -import java.math.BigDecimal; -import java.sql.Timestamp; -import java.sql.Types; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/27 - * Company: www.dtstack.com - * - * @author maqi - */ -public abstract class DBSink implements RetractStreamTableSink, Serializable { - - protected String driverName; - - protected String dbURL; - - protected String userName; - - protected String password; - - protected String dbType; - - protected int batchInterval = 1; - - protected int[] sqlTypes; - - protected String tableName; - - protected String sql; - - protected List primaryKeys; - - protected String[] fieldNames; - - private TypeInformation[] fieldTypes; - - private int parallelism = -1; - - - public RichSinkFunction createJdbcSinkFunc() { - - if (driverName == null || dbURL == null || userName == null - || password == null || sqlTypes == null || tableName == null) { - throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + - " must not be null. please check it!!!"); - } - - RetractJDBCOutputFormat.JDBCOutputFormatBuilder jdbcFormatBuild = RetractJDBCOutputFormat.buildJDBCOutputFormat(); - jdbcFormatBuild.setDBUrl(dbURL); - jdbcFormatBuild.setDrivername(driverName); - jdbcFormatBuild.setUsername(userName); - jdbcFormatBuild.setPassword(password); - jdbcFormatBuild.setInsertQuery(sql); - jdbcFormatBuild.setBatchInterval(batchInterval); - jdbcFormatBuild.setSqlTypes(sqlTypes); - jdbcFormatBuild.setTableName(tableName); - jdbcFormatBuild.setDBType(dbType); - jdbcFormatBuild.setDBSink(this); - - RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); - - OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); - return outputFormatSinkFunc; - } - - /** - * By now specified class type conversion. - * FIXME Follow-up has added a new type of time needs to be modified - * - * @param fieldTypeArray - */ - protected void buildSqlTypes(List fieldTypeArray) { - - int[] tmpFieldsType = new int[fieldTypeArray.size()]; - for (int i = 0; i < fieldTypeArray.size(); i++) { - String fieldType = fieldTypeArray.get(i).getName(); - if (fieldType.equals(Integer.class.getName())) { - tmpFieldsType[i] = Types.INTEGER; - } else if (fieldType.equals(Long.class.getName())) { - tmpFieldsType[i] = Types.BIGINT; - } else if (fieldType.equals(Byte.class.getName())) { - tmpFieldsType[i] = Types.TINYINT; - } else if (fieldType.equals(Short.class.getName())) { - tmpFieldsType[i] = Types.SMALLINT; - } else if (fieldType.equals(String.class.getName())) { - tmpFieldsType[i] = Types.CHAR; - } else if (fieldType.equals(Byte.class.getName())) { - tmpFieldsType[i] = Types.BINARY; - } else if (fieldType.equals(Float.class.getName())) { - tmpFieldsType[i] = Types.FLOAT; - } else if (fieldType.equals(Double.class.getName())) { - tmpFieldsType[i] = Types.DOUBLE; - } else if (fieldType.equals(Timestamp.class.getName())) { - tmpFieldsType[i] = Types.TIMESTAMP; - } else if (fieldType.equals(BigDecimal.class.getName())) { - tmpFieldsType[i] = Types.DECIMAL; - } else { - throw new RuntimeException("no support field type for sql. the input type:" + fieldType); - } - } - - this.sqlTypes = tmpFieldsType; - } - - /** - * Set the default frequency submit updated every submission - * - * @param batchInterval - */ - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; - } - - @Override - public void emitDataStream(DataStream> dataStream) { - RichSinkFunction richSinkFunction = createJdbcSinkFunc(); - DataStreamSink streamSink = dataStream.addSink(richSinkFunction); - streamSink.name(tableName); - if (parallelism > 0) { - streamSink.setParallelism(parallelism); - } - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); - } - - @Override - public TypeInformation getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - - public void setParallelism(int parallelism) { - this.parallelism = parallelism; - } - - public void setDbType(String dbType) { - this.dbType = dbType; - } - - /** - * you need to implements this method in your own class. - * - * @param tableName - * @param fields - */ - public abstract void buildSql(String tableName, List fields); - - /** - * sqlserver and oracle maybe implement - * - * @param tableName - * @param fieldNames - * @param realIndexes - * @return - */ - public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); -} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 586d4fb85..b4edd38c7 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -17,12 +17,32 @@ */ package com.dtstack.flink.sql.sink.rdb; +import com.dtstack.flink.sql.enums.EDatabaseType; import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.format.OracleOutputFormat; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import com.dtstack.flink.sql.sink.rdb.format.SqlserverOutputFormat; import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.sql.Types; import java.util.Arrays; import java.util.List; +import java.util.Map; /** * Reason: @@ -31,7 +51,67 @@ * * @author maqi */ -public abstract class RdbSink extends DBSink implements IStreamSinkGener { +public abstract class RdbSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + + protected String driverName; + + protected String dbURL; + + protected String userName; + + protected String password; + + protected String dbType; + + protected int batchInterval = 1; + + protected int[] sqlTypes; + + protected String tableName; + + protected String sql; + + protected List primaryKeys; + + protected String[] fieldNames; + + private TypeInformation[] fieldTypes; + + private int parallelism = -1; + + public RichSinkFunction createJdbcSinkFunc() { + if (driverName == null || dbURL == null || userName == null + || password == null || sqlTypes == null || tableName == null) { + throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + + " must not be null. please check it!!!"); + } + RetractJDBCOutputFormat outputFormat = getOutputFormat(dbType); + outputFormat.setDbURL(dbURL); + outputFormat.setDrivername(driverName); + outputFormat.setUsername(userName); + outputFormat.setPassword(password); + outputFormat.setInsertQuery(sql); + outputFormat.setBatchInterval(batchInterval); + outputFormat.setTypesArray(sqlTypes); + outputFormat.setTableName(tableName); + outputFormat.setDbType(dbType); + outputFormat.setDbSink(this); + + outputFormat.verifyField(); + OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); + return outputFormatSinkFunc; + } + + protected RetractJDBCOutputFormat getOutputFormat(String dbType) { + if (dbType.equalsIgnoreCase(EDatabaseType.SQLSERVER.name())) { + return new SqlserverOutputFormat(); + } else if (dbType.equalsIgnoreCase(EDatabaseType.ORACLE.name())) { + return new OracleOutputFormat(); + } else { + return new RetractJDBCOutputFormat(); + } + } + @Override public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { @@ -61,13 +141,126 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.password = tmpPassword; this.tableName = tmpTableName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); - this.dbType=rdbTableInfo.getType(); + this.dbType = rdbTableInfo.getType(); buildSql(tableName, fields); buildSqlTypes(fieldTypeArray); return this; } - public abstract String getDriverName(); + /** + * By now specified class type conversion. + * FIXME Follow-up has added a new type of time needs to be modified + * + * @param fieldTypeArray + */ + protected void buildSqlTypes(List fieldTypeArray) { + + int[] tmpFieldsType = new int[fieldTypeArray.size()]; + for (int i = 0; i < fieldTypeArray.size(); i++) { + String fieldType = fieldTypeArray.get(i).getName(); + if (fieldType.equals(Integer.class.getName())) { + tmpFieldsType[i] = Types.INTEGER; + } else if (fieldType.equals(Long.class.getName())) { + tmpFieldsType[i] = Types.BIGINT; + } else if (fieldType.equals(Byte.class.getName())) { + tmpFieldsType[i] = Types.TINYINT; + } else if (fieldType.equals(Short.class.getName())) { + tmpFieldsType[i] = Types.SMALLINT; + } else if (fieldType.equals(String.class.getName())) { + tmpFieldsType[i] = Types.CHAR; + } else if (fieldType.equals(Byte.class.getName())) { + tmpFieldsType[i] = Types.BINARY; + } else if (fieldType.equals(Float.class.getName())) { + tmpFieldsType[i] = Types.FLOAT; + } else if (fieldType.equals(Double.class.getName())) { + tmpFieldsType[i] = Types.DOUBLE; + } else if (fieldType.equals(Timestamp.class.getName())) { + tmpFieldsType[i] = Types.TIMESTAMP; + } else if (fieldType.equals(BigDecimal.class.getName())) { + tmpFieldsType[i] = Types.DECIMAL; + } else { + throw new RuntimeException("no support field type for sql. the input type:" + fieldType); + } + } + + this.sqlTypes = tmpFieldsType; + } + + + @Override + public void emitDataStream(DataStream> dataStream) { + RichSinkFunction richSinkFunction = createJdbcSinkFunc(); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + streamSink.name(tableName); + if (parallelism > 0) { + streamSink.setParallelism(parallelism); + } + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + /** + * Set the default frequency submit updated every submission + * + * @param batchInterval + */ + public void setBatchInterval(int batchInterval) { + this.batchInterval = batchInterval; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + public void setDbType(String dbType) { + this.dbType = dbType; + } + + /** + * you need to implements this method in your own class. + * + * @param tableName + * @param fields + */ + public abstract void buildSql(String tableName, List fields); + + /** + * sqlserver and oracle maybe implement + * + * @param tableName + * @param fieldNames + * @param realIndexes + * @return + */ + public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); + + public abstract String getDriverName(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java new file mode 100644 index 000000000..a9d635bca --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java @@ -0,0 +1,115 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.format; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleOutputFormat extends RetractJDBCOutputFormat { + + private final static String GET_ORACLE_INDEX_SQL = "SELECT " + + "t.INDEX_NAME," + + "t.COLUMN_NAME " + + "FROM " + + "user_ind_columns t," + + "user_indexes i " + + "WHERE " + + "t.index_name = i.index_name " + + "AND i.uniqueness = 'UNIQUE' " + + "AND t.table_name = '%s'"; + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + + PreparedStatement ps = getDbConn().prepareStatement(String.format(GET_ORACLE_INDEX_SQL, getTableName())); + ResultSet rs = ps.executeQuery(); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java similarity index 66% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index ee1d03abc..8f5329796 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.dtstack.flink.sql.sink.rdb; +package com.dtstack.flink.sql.sink.rdb.format; -import com.dtstack.flink.sql.enums.EDatabaseType; +import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -28,6 +28,7 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.sql.*; import java.util.*; import java.io.IOException; @@ -50,35 +51,23 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String password; private String drivername; private String dbURL; - private String insertQuery; private String tableName; private String dbType; + private RdbSink dbSink; private int batchInterval = 5000; + private String insertQuery; + public int[] typesArray; private Connection dbConn; private PreparedStatement upload; + + private int batchCount = 0; + //index field private Map> realIndexes = Maps.newHashMap(); //full field private List fullField = Lists.newArrayList(); - private DBSink dbSink; - - private int batchCount = 0; - - public int[] typesArray; - - private final static String GET_ORACLE_INDEX_SQL = "SELECT " + - "t.INDEX_NAME," + - "t.COLUMN_NAME " + - "FROM " + - "user_ind_columns t," + - "user_indexes i " + - "WHERE " + - "t.index_name = i.index_name " + - "AND i.uniqueness = 'UNIQUE' " + - "AND t.table_name = '%s'"; - public RetractJDBCOutputFormat() { } @@ -100,8 +89,8 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { - if (!EDatabaseType.MYSQL.name().equalsIgnoreCase(dbType) && isReplaceInsertQuery()) { - insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.fieldNames), realIndexes, fullField); + if (isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); } upload = dbConn.prepareStatement(insertQuery); } else { @@ -115,21 +104,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } } - private boolean isReplaceInsertQuery() throws SQLException { - getRealIndexes(); - getFullColumns(); - - if (!realIndexes.isEmpty()) { - for (List value : realIndexes.values()) { - for (String fieldName : dbSink.getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); @@ -307,153 +281,91 @@ public void close() throws IOException { } } - /** - * get db all index - * - * @throws SQLException - */ - public void getRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - ResultSet rs; - if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_ORACLE_INDEX_SQL, tableName)); - rs = ps.executeQuery(); - } else { - rs = dbConn.getMetaData().getIndexInfo(null, null, tableName, true, false); - } - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - realIndexes.put(k, v); - } - } + public boolean isReplaceInsertQuery() throws SQLException { + return false; } - /** - * get db all column name - * - * @throws SQLException - */ - public void getFullColumns() throws SQLException { - String schema = null; - if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { - String[] parts = tableName.split("\\."); - if (parts.length == 2) { - schema = parts[0].toUpperCase(); - tableName = parts[1]; - } + public void verifyField() { + if (StringUtils.isBlank(username)) { + LOG.info("Username was not supplied separately."); } - - ResultSet rs = dbConn.getMetaData().getColumns(null, schema, tableName, null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - fullField.add(columnName.toUpperCase()); - } + if (StringUtils.isBlank(password)) { + LOG.info("Password was not supplied separately."); + } + if (StringUtils.isBlank(dbURL)) { + throw new IllegalArgumentException("No dababase URL supplied."); + } + if (StringUtils.isBlank(insertQuery)) { + throw new IllegalArgumentException("No insertQuery suplied"); + } + if (StringUtils.isBlank(drivername)) { + throw new IllegalArgumentException("No driver supplied"); } - } - public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { - return new JDBCOutputFormatBuilder(); - } - public static class JDBCOutputFormatBuilder { - private final RetractJDBCOutputFormat format; + public void setUsername(String username) { + this.username = username; + } - protected JDBCOutputFormatBuilder() { - this.format = new RetractJDBCOutputFormat(); - } + public void setPassword(String password) { + this.password = password; + } - public JDBCOutputFormatBuilder setUsername(String username) { - format.username = username; - return this; - } + public void setDrivername(String drivername) { + this.drivername = drivername; + } - public JDBCOutputFormatBuilder setPassword(String password) { - format.password = password; - return this; - } + public void setDbURL(String dbURL) { + this.dbURL = dbURL; + } - public JDBCOutputFormatBuilder setDrivername(String drivername) { - format.drivername = drivername; - return this; - } + public void setTableName(String tableName) { + this.tableName = tableName; + } - public JDBCOutputFormatBuilder setDBUrl(String dbURL) { - format.dbURL = dbURL; - return this; - } + public void setDbType(String dbType) { + this.dbType = dbType; + } - public JDBCOutputFormatBuilder setInsertQuery(String query) { - format.insertQuery = query; - return this; - } + public void setDbSink(RdbSink dbSink) { + this.dbSink = dbSink; + } + public void setBatchInterval(int batchInterval) { + this.batchInterval = batchInterval; + } - public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { - format.batchInterval = batchInterval; - return this; - } + public void setInsertQuery(String insertQuery) { + this.insertQuery = insertQuery; + } - public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { - format.typesArray = typesArray; - return this; - } + public void setTypesArray(int[] typesArray) { + this.typesArray = typesArray; + } - public JDBCOutputFormatBuilder setTableName(String tableName) { - format.tableName = tableName; - return this; - } + public String getDbType() { + return dbType; + } - public JDBCOutputFormatBuilder setDBSink(DBSink dbSink) { - format.dbSink = dbSink; - return this; - } + public RdbSink getDbSink() { + return dbSink; + } - public JDBCOutputFormatBuilder setDBType(String dbType) { - format.dbType = dbType; - return this; - } + public Connection getDbConn() { + return dbConn; + } + public String getTableName() { + return tableName; + } - /** - * Finalizes the configuration and checks validity. - * - * @return Configured RetractJDBCOutputFormat - */ - public RetractJDBCOutputFormat finish() { - if (format.username == null) { - LOG.info("Username was not supplied separately."); - } - if (format.password == null) { - LOG.info("Password was not supplied separately."); - } - if (format.dbURL == null) { - throw new IllegalArgumentException("No dababase URL supplied."); - } - if (format.insertQuery == null) { - throw new IllegalArgumentException("No insertQuery suplied"); - } - if (format.drivername == null) { - throw new IllegalArgumentException("No driver supplied"); - } - return format; - } + public Map> getRealIndexes() { + return realIndexes; } + public List getFullField() { + return fullField; + } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java new file mode 100644 index 000000000..612bf7f17 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java @@ -0,0 +1,102 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.format; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverOutputFormat extends RetractJDBCOutputFormat { + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + String schema = null; + ResultSet rs = getDbConn().getMetaData().getColumns(null, schema, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 164413216..dbd75635e 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.sqlserver; +import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; @@ -29,7 +30,7 @@ * * @author maqi */ -public class SqlserverSink extends RdbSink { +public class SqlserverSink extends RdbSink implements IStreamSinkGener { private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; @Override @@ -97,7 +98,7 @@ public String getUpdateSql(List column, List fullColumn, String String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0 ) { + if (keyCols == null || keyCols.size() == 0) { continue; } if (fullColumn == null || column.contains(col)) { From f4871822249e4140d363fd3d0647f054deba0833 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 1 Dec 2018 12:17:29 +0800 Subject: [PATCH 42/42] modify structure --- .../flink/sql/sink/mysql/MysqlSink.java | 6 + .../dtstack/flink/sql/sink/rdb/RdbSink.java | 15 +-- .../sink/rdb/format/OracleOutputFormat.java | 115 ------------------ .../sqlserver}/SqlserverOutputFormat.java | 12 +- .../sql/sink/sqlserver/SqlserverSink.java | 6 + 5 files changed, 21 insertions(+), 133 deletions(-) delete mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java rename {rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format => sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver}/SqlserverOutputFormat.java (94%) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index 007c9a262..6d4d86d23 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import java.util.List; import java.util.Map; @@ -40,6 +41,11 @@ public class MysqlSink extends RdbSink implements IStreamSinkGener { public MysqlSink() { } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + @Override public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index b4edd38c7..d587639aa 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -17,11 +17,8 @@ */ package com.dtstack.flink.sql.sink.rdb; -import com.dtstack.flink.sql.enums.EDatabaseType; import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.sink.rdb.format.OracleOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; -import com.dtstack.flink.sql.sink.rdb.format.SqlserverOutputFormat; import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -85,7 +82,7 @@ public RichSinkFunction createJdbcSinkFunc() { throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + " must not be null. please check it!!!"); } - RetractJDBCOutputFormat outputFormat = getOutputFormat(dbType); + RetractJDBCOutputFormat outputFormat = getOutputFormat(); outputFormat.setDbURL(dbURL); outputFormat.setDrivername(driverName); outputFormat.setUsername(userName); @@ -102,15 +99,7 @@ public RichSinkFunction createJdbcSinkFunc() { return outputFormatSinkFunc; } - protected RetractJDBCOutputFormat getOutputFormat(String dbType) { - if (dbType.equalsIgnoreCase(EDatabaseType.SQLSERVER.name())) { - return new SqlserverOutputFormat(); - } else if (dbType.equalsIgnoreCase(EDatabaseType.ORACLE.name())) { - return new OracleOutputFormat(); - } else { - return new RetractJDBCOutputFormat(); - } - } + public abstract RetractJDBCOutputFormat getOutputFormat(); @Override diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java deleted file mode 100644 index a9d635bca..000000000 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * 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 com.dtstack.flink.sql.sink.rdb.format; - -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/30 - * Company: www.dtstack.com - * - * @author maqi - */ -public class OracleOutputFormat extends RetractJDBCOutputFormat { - - private final static String GET_ORACLE_INDEX_SQL = "SELECT " + - "t.INDEX_NAME," + - "t.COLUMN_NAME " + - "FROM " + - "user_ind_columns t," + - "user_indexes i " + - "WHERE " + - "t.index_name = i.index_name " + - "AND i.uniqueness = 'UNIQUE' " + - "AND t.table_name = '%s'"; - - - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); - - if (!getRealIndexes().isEmpty()) { - for (List value : getRealIndexes().values()) { - for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } - - /** - * get db all index - * - * @throws SQLException - */ - public void fillRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - - PreparedStatement ps = getDbConn().prepareStatement(String.format(GET_ORACLE_INDEX_SQL, getTableName())); - ResultSet rs = ps.executeQuery(); - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); - } - } - } - - /** - * get db all column name - * - * @throws SQLException - */ - public void fillFullColumns() throws SQLException { - ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); - } - } - } - - -} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java similarity index 94% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java rename to sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java index 612bf7f17..21d1f3bd1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -6,17 +6,19 @@ * 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 - *

+ * + * 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 com.dtstack.flink.sql.sink.rdb.format; +package com.dtstack.flink.sql.sink.sqlserver; + +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import java.sql.ResultSet; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index dbd75635e..1e266e3bc 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -19,6 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; import java.util.*; @@ -38,6 +39,11 @@ public String getDriverName() { return SQLSERVER_DRIVER; } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new SqlserverOutputFormat(); + } + @Override public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields);