From 44e423bf02040f28c3cd270785b18b39c1f927d3 Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Sun, 21 Oct 2018 22:16:25 +0800 Subject: [PATCH 01/78] on yarn job mode --- .../sql/launcher/ClusterClientFactory.java | 89 +++++++++++-------- 1 file changed, 53 insertions(+), 36 deletions(-) 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 aa8b5db5f..39bc01bbb 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 @@ -19,11 +19,8 @@ package com.dtstack.flink.sql.launcher; import org.apache.commons.lang.StringUtils; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; 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; import org.apache.flink.configuration.GlobalConfiguration; @@ -37,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; @@ -48,6 +46,8 @@ import java.util.Map; import java.util.Set; import com.dtstack.flink.sql.ClusterMode; +import org.apache.hadoop.yarn.exceptions.YarnException; +import java.io.IOException; /** * The Factory of ClusterClient @@ -61,8 +61,8 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarn.name())) { - return createYarnClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarn.name()) || mode.equals(ClusterMode.yarnPer.name())) { + return createYarnClient(launcherOptions,mode); } throw new IllegalArgumentException("Unsupported cluster client type: "); } @@ -79,7 +79,7 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio return clusterClient; } - public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { + public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); @@ -112,39 +112,16 @@ public boolean accept(File dir, String name) { 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(StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + ApplicationId applicationId = null; + if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode + applicationId = getYarnClusterApplicationId(yarnClient); + } else {//on yarn job mode + applicationId = createApplication(yarnClient); } + System.out.println("applicationId="+applicationId.toString()); + yarnClient.stop(); AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); @@ -161,6 +138,46 @@ public boolean accept(File dir, String name) { throw new UnsupportedOperationException("Haven't been developed yet!"); } + private static ApplicationId createApplication(YarnClient yarnClient)throws IOException, YarnException { + YarnClientApplication app = yarnClient.createApplication(); + return app.getApplicationSubmissionContext().getApplicationId(); + } + private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient) throws Exception{ + 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(StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); + } + return applicationId; + } + /** * 处理yarn HA的配置项 */ From bcdeaa21cdc5aa768a2c151c8346c388726101fe Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Mon, 29 Oct 2018 19:57:43 +0800 Subject: [PATCH 02/78] flink1.6+perjob --- bin/create.txt | 35 ++ core/pom.xml | 12 + .../flink/sql/MyLocalStreamEnvironment.java | 2 +- .../com/dtstack/flink/yarn/JobParameter.java | 130 ++++++ .../flink/yarn/YarnClusterConfiguration.java | 84 ++++ .../flink/yarn/YarnClusterDescriptor.java | 393 ++++++++++++++++++ .../sql/launcher/ClusterClientFactory.java | 150 ++++--- .../flink/sql/launcher/LauncherMain.java | 8 + pom.xml | 2 +- 9 files changed, 754 insertions(+), 62 deletions(-) create mode 100644 bin/create.txt create mode 100644 core/src/main/java/com/dtstack/flink/yarn/JobParameter.java create mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java create mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java diff --git a/bin/create.txt b/bin/create.txt new file mode 100644 index 000000000..819de4c41 --- /dev/null +++ b/bin/create.txt @@ -0,0 +1,35 @@ +CREATE TABLE MyTable( + name string, + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka10', + bootstrapServers ='10.148.9.92:9092', + zookeeperQuorum ='10.148.9.92:2181/kafka_test_1.0', + offsetReset ='latest', + topic ='flume_sink_test', + parallelism ='1' + ); + +CREATE TABLE MyResult( + channel VARCHAR, + name VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://10.9.14.120:3306/test?charset=utf8&useSSL=true', + userName ='test', + password ='test123', + tableName ='MyResult', + parallelism ='1' + ); + +insert +into + MyResult +select + channel, + name +from + MyTable; diff --git a/core/pom.xml b/core/pom.xml index 1040fcea6..851566fe0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -56,6 +56,18 @@ flink-streaming-scala_2.11 ${flink.version} + + + org.apache.flink + flink-shaded-hadoop2 + ${flink.version} + + + + org.apache.flink + flink-yarn_2.11 + ${flink.version} + diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java index 54ddaa647..f41ecf00b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java @@ -100,7 +100,7 @@ public JobExecutionResult execute(String jobName) throws Exception { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -1L); + configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "-1L"); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined diff --git a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java b/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java new file mode 100644 index 000000000..1879c6ea1 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java @@ -0,0 +1,130 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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.yarn; + +import java.util.Objects; +import java.util.Properties; + +public class JobParameter +{ + private int parallelism = 1; + private String queue = "default"; + private int taskManagerMemoryMb = 1024; + private int taskManagerCount = 1; + private int taskManagerSlots = 1; + private int jobManagerMemoryMb = 1024; + + public JobParameter() {} + + public JobParameter(Properties confProperties) { + this.parallelism = confProperties.getProperty("parallelism")==null?parallelism:Integer.parseInt(confProperties.getProperty("parallelism")); + this.queue = confProperties.getProperty("queue")==null?queue:confProperties.getProperty("queue"); + this.taskManagerMemoryMb = confProperties.getProperty("taskManagerMemoryMb")==null?taskManagerMemoryMb:Integer.parseInt(confProperties.getProperty("taskManagerMemoryMb")); + this.taskManagerCount = confProperties.getProperty("taskManagerCount")==null?taskManagerCount:Integer.parseInt(confProperties.getProperty("taskManagerCount")); + this.taskManagerSlots = confProperties.getProperty("taskManagerSlots")==null?taskManagerSlots:Integer.parseInt(confProperties.getProperty("taskManagerSlots")); + this.jobManagerMemoryMb = confProperties.getProperty("jobManagerMemoryMb")==null?jobManagerMemoryMb:Integer.parseInt(confProperties.getProperty("jobManagerMemoryMb")); + } + + public JobParameter(int parallelism, String queue, int taskManagerMemoryMb, int taskManagerCount, int taskManagerSlots, int jobManagerMemoryMb) { + this.parallelism = parallelism; + this.queue = queue; + this.taskManagerMemoryMb = taskManagerMemoryMb; + this.taskManagerCount = taskManagerCount; + this.taskManagerSlots = taskManagerSlots; + this.jobManagerMemoryMb = jobManagerMemoryMb; + } + + public void setQueue(String queue) + { + this.queue = queue; + } + + public void setTaskManagerCount(int taskManagerCount) + { + this.taskManagerCount = taskManagerCount; + } + + public void setTaskManagerMemoryMb(int taskManagerMemoryMb) + { + this.taskManagerMemoryMb = taskManagerMemoryMb; + } + + public void setTaskManagerSlots(int taskManagerSlots) + { + this.taskManagerSlots = taskManagerSlots; + } + + public void setJobManagerMemoryMb(int jobManagerMemoryMb) + { + this.jobManagerMemoryMb = jobManagerMemoryMb; + } + + public void setParallelism(int parallelism) + { + this.parallelism = parallelism; + } + + public int getParallelism() + { + return parallelism; + } + + public String getQueue() + { + return queue; + } + + public int getJobManagerMemoryMb() + { + return jobManagerMemoryMb; + } + + public int getTaskManagerSlots() + { + return taskManagerSlots; + } + + public int getTaskManagerCount() + { + return taskManagerCount; + } + + public int getTaskManagerMemoryMb() + { + return taskManagerMemoryMb; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JobParameter jobParameter = (JobParameter) o; + return Objects.equals(this.queue, jobParameter.queue) && + Objects.equals(this.taskManagerCount, jobParameter.taskManagerCount) && + Objects.equals(this.taskManagerMemoryMb, jobParameter.taskManagerMemoryMb); + } + + @Override + public int hashCode() + { + return Objects.hash(queue, taskManagerMemoryMb, taskManagerCount); + } +} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java new file mode 100644 index 000000000..f2ccea2c4 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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.yarn; + +import org.apache.flink.configuration.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.util.Set; + +public class YarnClusterConfiguration { + /** + * The configuration used by YARN (i.e.,
yarn-site.xml
). + */ + private final YarnConfiguration yarnConf; + + /** + * The home directory of all job where all the temporary files for each jobs are stored. + */ + private final String appRootDir; + + /** + * The location of the Flink jar. + */ + private final Path flinkJar; + + /** + * Additional resources to be localized for both JobManager and TaskManager. + * They will NOT be added into the classpaths. + */ + private final Set resourcesToLocalize; + + /** + * flink conf + */ + private final Configuration flinkConfiguration; + + public YarnClusterConfiguration( + Configuration flinkConf, + YarnConfiguration conf, + String appRootDir, + Path flinkJar, + Set resourcesToLocalize) { + this.flinkConfiguration = flinkConf; + this.yarnConf = conf; + this.appRootDir = appRootDir; + this.flinkJar = flinkJar; + this.resourcesToLocalize = resourcesToLocalize; + } + + YarnConfiguration yarnConf() { + return yarnConf; + } + + public String appRootDir() { + return appRootDir; + } + + public Configuration flinkConfiguration() { + return flinkConfiguration; + } + + public Path flinkJar() { + return flinkJar; + } + + public Set resourcesToLocalize() { + return resourcesToLocalize; + } + +} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java new file mode 100644 index 000000000..d442d16f2 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java @@ -0,0 +1,393 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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.yarn; + +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.*; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.*; +import java.util.concurrent.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.yarn.api.records.YarnApplicationState.NEW; + +public class YarnClusterDescriptor + extends AbstractYarnClusterDescriptor +{ + private static final String APPLICATION_TYPE = "58_FLINK"; + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + private static final int MAX_ATTEMPT = 1; + private static final long DEPLOY_TIMEOUT_MS = 600 * 1000; + private static final long RETRY_DELAY_MS = 250; + private static final ScheduledExecutorService YARN_POLL_EXECUTOR = Executors.newSingleThreadScheduledExecutor(); + + private final YarnClusterConfiguration clusterConf; + private final YarnClient yarnClient; + private final JobParameter appConf; + private final Path homedir; + private final ApplicationId yarnAppId; + private final String jobName; + private final Iterable userProvidedJars; + private Path flinkJar; + + public YarnClusterDescriptor( + final YarnClusterConfiguration clusterConf, + final YarnClient yarnClient, + final JobParameter appConf, + ApplicationId yarnAppId, + String jobName, + Iterable userProvidedJars) + { + super(clusterConf.flinkConfiguration(), clusterConf.yarnConf(), clusterConf.appRootDir(), yarnClient, false); + this.jobName = jobName; + this.clusterConf = clusterConf; + this.yarnClient = yarnClient; + this.appConf = appConf; + this.yarnAppId = yarnAppId; + this.userProvidedJars = userProvidedJars; + this.homedir = new Path(clusterConf.appRootDir(), yarnAppId.toString()); + } + + @Override + protected String getYarnSessionClusterEntrypoint() + { + return YarnApplicationMasterRunner.class.getName(); + } + + /** + * 提交到yarn时 任务启动入口类 + */ + @Override + protected String getYarnJobClusterEntrypoint() + { + return YarnApplicationMasterRunner.class.getName(); + } + + @Override + protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) + throws Exception + { + return new RestClusterClient<>( + flinkConfiguration, + report.getApplicationId()); + } + + @Override + public YarnClient getYarnClient() + { + return this.yarnClient; + } + + public YarnClusterClient deploy() + { + ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); + context.setApplicationId(yarnAppId); + try { + ApplicationReport report = startAppMaster(context); + + Configuration conf = getFlinkConfiguration(); + conf.setString(JobManagerOptions.ADDRESS.key(), report.getHost()); + conf.setInteger(JobManagerOptions.PORT.key(), report.getRpcPort()); + + return new YarnClusterClient(this, + appConf.getTaskManagerCount(), + appConf.getTaskManagerSlots(), + report, conf, false); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private ApplicationReport startAppMaster(ApplicationSubmissionContext appContext) + throws Exception + { + ApplicationId appId = appContext.getApplicationId(); + appContext.setMaxAppAttempts(MAX_ATTEMPT); + + Map localResources = new HashMap<>(); + Set shippedPaths = new HashSet<>(); + collectLocalResources(localResources, shippedPaths); + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( + getYarnJobClusterEntrypoint(), + false, + true, + false, + appConf.getJobManagerMemoryMb() + ); + + amContainer.setLocalResources(localResources); + + final String classPath = String.join(File.pathSeparator, localResources.keySet()); + + final String shippedFiles = shippedPaths.stream() + .map(path -> path.getName() + "=" + path) + .collect(Collectors.joining(",")); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = setUpAmEnvironment( + appId, + classPath,shippedFiles, + //"","", + getDynamicPropertiesEncoded() + ); + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(appConf.getJobManagerMemoryMb()); //设置jobManneger + capability.setVirtualCores(1); //默认是1 + + appContext.setApplicationName(jobName); + appContext.setApplicationType(APPLICATION_TYPE); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + //appContext.setApplicationTags(appConf.getAppTags()); + if (appConf.getQueue() != null) { + appContext.setQueue(appConf.getQueue()); + } + + LOG.info("Submitting application master {}", appId); + yarnClient.submitApplication(appContext); + + PollDeploymentStatus poll = new PollDeploymentStatus(appId); + YARN_POLL_EXECUTOR.submit(poll); + try { + return poll.result.get(); + } + catch (ExecutionException e) { + LOG.warn("Failed to deploy {}, cause: {}", appId.toString(), e.getCause()); + yarnClient.killApplication(appId); + throw (Exception) e.getCause(); + } + } + + private void collectLocalResources( + Map resources, + Set shippedPaths + ) + throws IOException, URISyntaxException + { + if(clusterConf.flinkJar() != null) { + Path flinkJar = clusterConf.flinkJar(); + LocalResource flinkJarResource = setupLocalResource(flinkJar, homedir, ""); //放到 Appid/根目录下 + this.flinkJar = ConverterUtils.getPathFromYarnURL(flinkJarResource.getResource()); + resources.put("flink.jar", flinkJarResource); + } + if(clusterConf.resourcesToLocalize() != null) { + for (Path p : clusterConf.resourcesToLocalize()) { //主要是 flink.jar log4f.propors 和 flink.yaml 三个文件 + LocalResource resource = setupLocalResource(p, homedir, ""); //这些需要放到根目录下 + resources.put(p.getName(), resource); + if ("log4j.properties".equals(p.getName())) { + shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); + } + } + } + if(userProvidedJars != null) { + for (Path p : userProvidedJars) { + String name = p.getName(); + if (resources.containsKey(name)) { //这里当jar 有重复的时候 会抛出异常 + LOG.warn("Duplicated name in the shipped files {}", p); + } else { + LocalResource resource = setupLocalResource(p, homedir, "jars"); //这些放到 jars目录下 + resources.put(name, resource); + shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); + } + } + } + } + + private LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) + throws IOException + { + LocalResource localResource = Records.newRecord(LocalResource.class); + FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); + localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); + localResource.setSize(jarStat.getLen()); + localResource.setTimestamp(jarStat.getModificationTime()); + localResource.setType(LocalResourceType.FILE); + localResource.setVisibility(LocalResourceVisibility.APPLICATION); + return localResource; + } + + private LocalResource setupLocalResource( + Path localSrcPath, + Path homedir, + String relativeTargetPath) + throws IOException + { + if (new File(localSrcPath.toUri().getPath()).isDirectory()) { + throw new IllegalArgumentException("File to copy must not be a directory: " + + localSrcPath); + } + + // copy resource to HDFS + String suffix = "." + (relativeTargetPath.isEmpty() ? "" : "/" + relativeTargetPath) + + "/" + localSrcPath.getName(); + + Path dst = new Path(homedir, suffix); + + LOG.info("Uploading {}", dst); + + FileSystem hdfs = FileSystem.get(clusterConf.yarnConf()); + hdfs.copyFromLocalFile(false, true, localSrcPath, dst); + + // now create the resource instance + LocalResource resource = registerLocalResource(hdfs, dst); + return resource; + } + + private Map setUpAmEnvironment( + ApplicationId appId, + String amClassPath, + String shipFiles, + String dynamicProperties) + throws IOException, URISyntaxException + { + final Map appMasterEnv = new HashMap<>(); + + // set Flink app class path + appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, amClassPath); + + // set Flink on YARN internal configuration values + appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(appConf.getTaskManagerCount())); + appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(appConf.getTaskManagerMemoryMb())); + appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(appConf.getTaskManagerSlots())); + appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, flinkJar.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homedir.toString()); //$home/.flink/appid 这个目录里面存放临时数据 + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, shipFiles); + + appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(true)); //是否分离 分离就cluser模式 否则是client模式 + + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, + UserGroupInformation.getCurrentUser().getUserName()); + + if (dynamicProperties != null) { + appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicProperties); + } + + // set classpath from YARN configuration + Utils.setupYarnClassPath(clusterConf.yarnConf(), appMasterEnv); + + return appMasterEnv; + } + + /** + * flink 1.5 add + */ + @Override + public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) + throws ClusterDeploymentException + { + throw new UnsupportedOperationException("this method have't support!"); + } + + private final class PollDeploymentStatus + implements Runnable + { + private final CompletableFuture result = new CompletableFuture<>(); + private final ApplicationId appId; + private YarnApplicationState lastAppState = NEW; + private long startTime; + + private PollDeploymentStatus(ApplicationId appId) + { + this.appId = appId; + } + + @Override + public void run() + { + if (startTime == 0) { + startTime = System.currentTimeMillis(); + } + + try { + ApplicationReport report = poll(); + if (report == null) { + YARN_POLL_EXECUTOR.schedule(this, RETRY_DELAY_MS, TimeUnit.MILLISECONDS); + } + else { + result.complete(report); + } + } + catch (YarnException | IOException e) { + result.completeExceptionally(e); + } + } + + private ApplicationReport poll() + throws IOException, YarnException + { + ApplicationReport report; + report = yarnClient.getApplicationReport(appId); + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + + switch (appState) { + case FAILED: + case FINISHED: + //TODO: the finished state may be valid in flip-6 + case KILLED: + throw new IOException("The YARN application unexpectedly switched to state " + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + appId); + //break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + lastAppState = appState; + if (System.currentTimeMillis() - startTime > DEPLOY_TIMEOUT_MS) { + throw new RuntimeException(String.format("Deployment took more than %d seconds. " + + "Please check if the requested resources are available in the YARN cluster", DEPLOY_TIMEOUT_MS)); + } + return null; + } + return report; + } + } +} 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 39bc01bbb..9bd160966 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,10 @@ package com.dtstack.flink.sql.launcher; +import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.yarn.JobParameter; +import com.dtstack.flink.yarn.YarnClusterConfiguration; +import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; @@ -30,6 +34,7 @@ import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; @@ -39,15 +44,16 @@ import java.io.File; import java.io.FilenameFilter; 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.Set; +import java.net.URLDecoder; +import java.util.*; + import com.dtstack.flink.sql.ClusterMode; import org.apache.hadoop.yarn.exceptions.YarnException; import java.io.IOException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Objects.requireNonNull; /** * The Factory of ClusterClient @@ -81,61 +87,112 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + Configuration flinkConf = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); - YarnConfiguration yarnConf = new YarnConfiguration(); + YarnConfiguration yarnConf; if(StringUtils.isNotBlank(yarnConfDir)) { try { - - config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); - FileSystem.initialize(config); + flinkConf.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + FileSystem.initialize(flinkConf); 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()); - } - } + yarnConf = loadYarnConfiguration(yarnConfDir); YarnClient yarnClient = YarnClient.createYarnClient(); haYarnConf(yarnConf); yarnClient.init(yarnConf); yarnClient.start(); + String confProp = launcherOptions.getConfProp(); + confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); + System.out.println("confProp="+confProp); + Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); + ApplicationId applicationId = null; + ClusterClient clusterClient = null; if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode applicationId = getYarnClusterApplicationId(yarnClient); - } else {//on yarn job mode + System.out.println("applicationId="+applicationId.toString()); + + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( + flinkConf, yarnConf, ".", yarnClient, false); + clusterClient = clusterDescriptor.retrieve(applicationId); + + System.out.println("applicationId="+applicationId.toString()+" has retrieve!"); + } else {//on yarn per-job mode applicationId = createApplication(yarnClient); - } - System.out.println("applicationId="+applicationId.toString()); + System.out.println("applicationId="+applicationId.toString()); - yarnClient.stop(); + YarnClusterConfiguration clusterConf = getYarnClusterConfiguration(flinkConf,yarnConf,flinkConfDir); + //jobmanager+taskmanager param + JobParameter appConf = new JobParameter(confProperties); - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); - ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + com.dtstack.flink.yarn.YarnClusterDescriptor clusterDescriptor = new com.dtstack.flink.yarn.YarnClusterDescriptor( + clusterConf, yarnClient, appConf,applicationId, launcherOptions.getName(),null ); + clusterClient = clusterDescriptor.deploy(); + + System.out.println("applicationId="+applicationId.toString()+" has deploy!"); + } clusterClient.setDetached(true); + yarnClient.stop(); return clusterClient; } } catch(Exception e) { throw new RuntimeException(e); } } + throw new UnsupportedOperationException("Haven't been developed yet!"); + } + + private static YarnConfiguration loadYarnConfiguration(String yarnConfDir) + { + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem"); + Stream.of("yarn-site.xml", "core-site.xml", "hdfs-site.xml").forEach(file -> { + File site = new File(requireNonNull(yarnConfDir, "ENV HADOOP_CONF_DIR is not setting"), file); + if (site.exists() && site.isFile()) { + hadoopConf.addResource(new org.apache.hadoop.fs.Path(site.toURI())); + } + else { + throw new RuntimeException(site + " not exists"); + } + }); + + YarnConfiguration yarnConf = new YarnConfiguration(hadoopConf); + // try (PrintWriter pw = new PrintWriter(new FileWriter(yarnSite))) { //write local file + // yarnConf.writeXml(pw); + // } + return yarnConf; + } + public static YarnClusterConfiguration getYarnClusterConfiguration(Configuration flinkConf,YarnConfiguration yarnConf,String flinkConfDir) + { + Path flinkJar = new Path(getFlinkJarFile(flinkConfDir).toURI()); + @SuppressWarnings("ConstantConditions") final Set resourcesToLocalize = Stream + .of("flink-conf.yaml", "log4j.properties") + .map(x -> new Path(new File(flinkConfDir, x).toURI())) + .collect(Collectors.toSet()); - throw new UnsupportedOperationException("Haven't been developed yet!"); + String home = "hdfs:///tmp/flink/apps"; + //String home = "hdfs://test-cluster/tmp/flink/apps"; + return new YarnClusterConfiguration( + flinkConf, + yarnConf, + home, + flinkJar, + resourcesToLocalize); + } + + public static final String FLINK_DIST = "flink-dist"; + private static File getFlinkJarFile(String flinkConfDir) + { + String errorMessage = "error not search " + FLINK_DIST + "*.jar"; + File[] files = requireNonNull(new File(flinkConfDir, "/../lib").listFiles(), errorMessage); + Optional file = Arrays.stream(files) + .filter(f -> f.getName().startsWith(FLINK_DIST)).findFirst(); + return file.orElseThrow(() -> new IllegalArgumentException(errorMessage)); } private static ApplicationId createApplication(YarnClient yarnClient)throws IOException, YarnException { @@ -198,31 +255,4 @@ private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop 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 55f085c98..957d8cb18 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 @@ -27,6 +27,9 @@ 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.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; @@ -64,8 +67,13 @@ public static void main(String[] args) throws Exception { if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); } + //final JobGraph jobGraph; + //jobGraph = PackagedProgramUtils.createJobGraph(program, new Configuration(), 1); + //clusterClient.runDetached(jobGraph,null); clusterClient.run(program, 1); clusterClient.shutdown(); + + System.exit(0); } } } diff --git a/pom.xml b/pom.xml index 0922f739b..9b8f3ea45 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ UTF-8 - 1.5.3 + 1.6.0 From 7522d5b4d4c80b1b6a819f0a7fcb307336dd0c63 Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Mon, 29 Oct 2018 20:14:42 +0800 Subject: [PATCH 03/78] flink1.6+perjob --- bin/create.txt | 35 ------------------- .../sql/launcher/ClusterClientFactory.java | 1 - 2 files changed, 36 deletions(-) delete mode 100644 bin/create.txt diff --git a/bin/create.txt b/bin/create.txt deleted file mode 100644 index 819de4c41..000000000 --- a/bin/create.txt +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE MyTable( - name string, - channel STRING, - pv INT, - xctime bigint, - CHARACTER_LENGTH(channel) AS timeLeng - )WITH( - type ='kafka10', - bootstrapServers ='10.148.9.92:9092', - zookeeperQuorum ='10.148.9.92:2181/kafka_test_1.0', - offsetReset ='latest', - topic ='flume_sink_test', - parallelism ='1' - ); - -CREATE TABLE MyResult( - channel VARCHAR, - name VARCHAR - )WITH( - type ='mysql', - url ='jdbc:mysql://10.9.14.120:3306/test?charset=utf8&useSSL=true', - userName ='test', - password ='test123', - tableName ='MyResult', - parallelism ='1' - ); - -insert -into - MyResult -select - channel, - name -from - MyTable; 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 9bd160966..36ab25d2a 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 @@ -176,7 +176,6 @@ public static YarnClusterConfiguration getYarnClusterConfiguration(Configuration .collect(Collectors.toSet()); String home = "hdfs:///tmp/flink/apps"; - //String home = "hdfs://test-cluster/tmp/flink/apps"; return new YarnClusterConfiguration( flinkConf, yarnConf, From 83edbc2bcd7da643a88fe4373445088ea2eccccd Mon Sep 17 00:00:00 2001 From: ambition119 <1269223860> Date: Mon, 12 Nov 2018 14:48:02 +0800 Subject: [PATCH 04/78] =?UTF-8?q?SQL=20CEP=20=E5=AE=9E=E7=8E=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/pom.xml | 25 +- .../flink/sql/parser/InsertSqlParser.java | 11 +- .../main/scala/com/dtstack/flink/App.scala | 11 + .../calcite/RelTimeIndicatorConverter.scala | 482 +++++ .../flink/table/codegen/CodeGenerator.scala | 1920 +++++++++++++++++ .../table/codegen/MatchCodeGenerator.scala | 574 +++++ .../flink/table/codegen/generated.scala | 100 + .../nodes/datastream/DataStreamMatch.scala | 309 +++ .../nodes/logical/FlinkLogicalMatch.scala | 115 + .../table/plan/rules/FlinkRuleSets.scala | 209 ++ .../datastream/DataStreamMatchRule.scala | 46 + .../table/runtime/cepmatch/ConvertToRow.scala | 14 + .../cepmatch/IterativeConditionRunner.scala | 40 + .../table/runtime/cepmatch/MatchUtil.scala | 99 + .../PatternFlatSelectFunctionRunner.scala | 47 + .../PatternSelectFunctionRunner.scala | 45 + .../table/validate/FunctionCatalog.scala | 534 +++++ .../flink/sql/side/SideSqlExecTest.java | 48 +- .../dtstack/flink/sql/sink/mysql/DBSink.java | 3 + 19 files changed, 4622 insertions(+), 10 deletions(-) create mode 100644 core/src/main/scala/com/dtstack/flink/App.scala create mode 100644 core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/generated.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala diff --git a/core/pom.xml b/core/pom.xml index 1040fcea6..ba5970f91 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,9 +28,9 @@ - org.apache.flink - flink-table_2.11 - ${flink.version} + joda-time + joda-time + 2.5 @@ -45,6 +45,25 @@ ${flink.version} + + org.apache.flink + flink-table_2.11 + ${flink.version} + + + + org.apache.calcite + calcite-server + + 1.16.0 + + + + org.apache.flink + flink-cep-scala_2.11 + ${flink.version} + + org.apache.flink flink-scala_2.11 diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e3d65f420..3e3a75880 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,12 +20,7 @@ package com.dtstack.flink.sql.parser; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; @@ -113,6 +108,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ sqlParseResult.addSourceTable(identifierNode.toString()); } break; + case MATCH_RECOGNIZE: + SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; + sqlParseResult.addSourceTable(node.getTableRef().toString()); + break; default: //do nothing break; diff --git a/core/src/main/scala/com/dtstack/flink/App.scala b/core/src/main/scala/com/dtstack/flink/App.scala new file mode 100644 index 000000000..e74ccc28a --- /dev/null +++ b/core/src/main/scala/com/dtstack/flink/App.scala @@ -0,0 +1,11 @@ +package com.dtstack.flink + +/** + * Hello world! + * + */ +object App { + def main(args: Array[String]): Unit = { + println( "Hello World!" ) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala new file mode 100644 index 000000000..e120addb3 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala @@ -0,0 +1,482 @@ +package org.apache.flink.table.calcite + +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core._ +import org.apache.calcite.rel.logical._ +import org.apache.calcite.rel.{RelNode, RelShuttle} +import org.apache.calcite.rex._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} +import org.apache.flink.table.functions.sql.ProctimeSqlFunction +import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate +import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType +import org.apache.flink.table.validate.BasicOperatorTable + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a + * time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in + * some cases, but not all. + */ +class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { + + private val timestamp = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) + + override def visit(intersect: LogicalIntersect): RelNode = + throw new TableException("Logical intersect in a stream environment is not supported yet.") + + override def visit(union: LogicalUnion): RelNode = { + // visit children and update inputs + val inputs = union.getInputs.map(_.accept(this)) + + // make sure that time indicator types match + val inputTypes = inputs.map(_.getRowType) + + val head = inputTypes.head.getFieldList.map(_.getType) + + val isValid = inputTypes.forall { t => + val fieldTypes = t.getFieldList.map(_.getType) + + fieldTypes.zip(head).forall { case (l, r) => + // check if time indicators match + if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { + val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime + val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime + leftTime == rightTime + } + // one side is not an indicator + else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { + false + } + // uninteresting types + else { + true + } + } + } + + if (!isValid) { + throw new ValidationException( + "Union fields with time attributes have different types.") + } + + LogicalUnion.create(inputs, union.all) + } + + override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate) + + override def visit(minus: LogicalMinus): RelNode = + throw new TableException("Logical minus in a stream environment is not supported yet.") + + override def visit(sort: LogicalSort): RelNode = { + + val input = sort.getInput.accept(this) + LogicalSort.create(input, sort.collation, sort.offset, sort.fetch) + } + + // override def visit(`match`: LogicalMatch): RelNode = + // throw new TableException("Logical match in a stream environment is not supported yet.") + // and cpe entry + override def visit(`match`: LogicalMatch): RelNode = { + val rowType = `match`.getInput.getRowType + + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + rowType.getFieldList.map(_.getType)) + + val patternDefinitions = + `match`.getPatternDefinitions.foldLeft(mutable.Map[String, RexNode]()) { + case (m, (k, v)) => m += k -> v.accept(materializer) + } + + val measures = `match`.getMeasures.foldLeft(mutable.Map[String, RexNode]()) { + case (m, (k, v)) => m += k -> v.accept(materializer) + } + + val outputTypeBuilder = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .builder() + + `match`.getRowType.getFieldList.asScala + .foreach(x => measures.get(x.getName) match { + case Some(measure) => outputTypeBuilder.add(x.getName, measure.getType) + case None => outputTypeBuilder.add(x) + }) + + LogicalMatch.create( + `match`.getInput, + outputTypeBuilder.build(), + `match`.getPattern, + `match`.isStrictStart, + `match`.isStrictEnd, + patternDefinitions, + measures, + `match`.getAfter, + `match`.getSubsets.asInstanceOf[java.util.Map[String, java.util.TreeSet[String]]], + `match`.isAllRows, + `match`.getPartitionKeys, + `match`.getOrderKeys, + `match`.getInterval) + } + + + override def visit(other: RelNode): RelNode = other match { + + case uncollect: Uncollect => + // visit children and update inputs + val input = uncollect.getInput.accept(this) + Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality) + + case scan: LogicalTableFunctionScan => + scan + + case aggregate: LogicalWindowAggregate => + val convAggregate = convertAggregate(aggregate) + + LogicalWindowAggregate.create( + aggregate.getWindow, + aggregate.getNamedProperties, + convAggregate) + + case _ => + throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") + } + + + override def visit(exchange: LogicalExchange): RelNode = + throw new TableException("Logical exchange in a stream environment is not supported yet.") + + override def visit(scan: TableScan): RelNode = scan + + override def visit(scan: TableFunctionScan): RelNode = + throw new TableException("Table function scan in a stream environment is not supported yet.") + + override def visit(values: LogicalValues): RelNode = values + + override def visit(filter: LogicalFilter): RelNode = { + // visit children and update inputs + val input = filter.getInput.accept(this) + + // We do not materialize time indicators in conditions because they can be locally evaluated. + // Some conditions are evaluated by special operators (e.g., time window joins). + // Time indicators in remaining conditions are materialized by Calc before the code generation. + LogicalFilter.create(input, filter.getCondition) + } + + override def visit(project: LogicalProject): RelNode = { + // visit children and update inputs + val input = project.getInput.accept(this) + + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + input.getRowType.getFieldList.map(_.getType)) + + val projects = project.getProjects.map(_.accept(materializer)) + val fieldNames = project.getRowType.getFieldNames + LogicalProject.create(input, projects, fieldNames) + } + + override def visit(join: LogicalJoin): RelNode = { + val left = join.getLeft.accept(this) + val right = join.getRight.accept(this) + + LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType) + + } + + + override def visit(correlate: LogicalCorrelate): RelNode = { + // visit children and update inputs + val inputs = correlate.getInputs.map(_.accept(this)) + + val right = inputs(1) match { + case scan: LogicalTableFunctionScan => + // visit children and update inputs + val scanInputs = scan.getInputs.map(_.accept(this)) + + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + inputs.head.getRowType.getFieldList.map(_.getType)) + + val call = scan.getCall.accept(materializer) + LogicalTableFunctionScan.create( + scan.getCluster, + scanInputs, + call, + scan.getElementType, + scan.getRowType, + scan.getColumnMappings) + + case _ => + inputs(1) + } + + LogicalCorrelate.create( + inputs.head, + right, + correlate.getCorrelationId, + correlate.getRequiredColumns, + correlate.getJoinType) + } + + + + + private def convertAggregate(aggregate: Aggregate): LogicalAggregate = { + // visit children and update inputs + val input = aggregate.getInput.accept(this) + + // add a project to materialize aggregation arguments/grouping keys + + val refIndices = mutable.Set[Int]() + + // check arguments of agg calls + aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) { + // count(*) has an empty argument list + (0 until input.getRowType.getFieldCount).foreach(refIndices.add) + } else { + // for other aggregations + call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add) + }) + + // check grouping sets + aggregate.getGroupSets.foreach(set => + set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add) + ) + + val needsMaterialization = refIndices.exists(idx => + isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType)) + + // create project if necessary + val projectedInput = if (needsMaterialization) { + + // insert or merge with input project if + // a time attribute is accessed and needs to be materialized + input match { + + // merge + case lp: LogicalProject => + val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) => + if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) { + if (isRowtimeIndicatorType(expr.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, expr) + } else { + // generate proctime access + rexBuilder.makeCall(ProctimeSqlFunction, expr) + } + } else { + expr + } + } + + LogicalProject.create( + lp.getInput, + projects, + input.getRowType.getFieldNames) + + // new project + case _ => + val projects = input.getRowType.getFieldList.map { field => + if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) { + if (isRowtimeIndicatorType(field.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast( + timestamp, + new RexInputRef(field.getIndex, field.getType)) + } else { + // generate proctime access + rexBuilder.makeCall( + ProctimeSqlFunction, + new RexInputRef(field.getIndex, field.getType)) + } + } else { + new RexInputRef(field.getIndex, field.getType) + } + } + + LogicalProject.create( + input, + projects, + input.getRowType.getFieldNames) + } + } else { + // no project necessary + input + } + + // remove time indicator type as agg call return type + val updatedAggCalls = aggregate.getAggCallList.map { call => + val callType = if (isTimeIndicatorType(call.getType)) { + timestamp + } else { + call.getType + } + AggregateCall.create( + call.getAggregation, + call.isDistinct, + call.getArgList, + call.filterArg, + callType, + call.name) + } + + LogicalAggregate.create( + projectedInput, + aggregate.indicator, + aggregate.getGroupSet, + aggregate.getGroupSets, + updatedAggCalls) + } + +} + +object RelTimeIndicatorConverter { + + def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = { + val converter = new RelTimeIndicatorConverter(rexBuilder) + val convertedRoot = rootRel.accept(converter) + + var needsConversion = false + + // materialize remaining proctime indicators + val projects = convertedRoot.getRowType.getFieldList.map(field => + if (isProctimeIndicatorType(field.getType)) { + needsConversion = true + rexBuilder.makeCall( + ProctimeSqlFunction, + new RexInputRef(field.getIndex, field.getType)) + } else { + new RexInputRef(field.getIndex, field.getType) + } + ) + + // add final conversion if necessary + if (needsConversion) { + LogicalProject.create( + convertedRoot, + projects, + convertedRoot.getRowType.getFieldNames) + } else { + convertedRoot + } + } + + /** + * Materializes time indicator accesses in an expression. + * + * @param expr The expression in which time indicators are materialized. + * @param rowType The input schema of the expression. + * @param rexBuilder A RexBuilder. + * + * @return The expression with materialized time indicators. + */ + def convertExpression(expr: RexNode, rowType: RelDataType, rexBuilder: RexBuilder): RexNode = { + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + rowType.getFieldList.map(_.getType)) + + expr.accept(materializer) + } +} + +class RexTimeIndicatorMaterializer( + private val rexBuilder: RexBuilder, + private val input: Seq[RelDataType]) + extends RexShuttle { + + private val timestamp = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) + + override def visitInputRef(inputRef: RexInputRef): RexNode = { + // reference is interesting + if (isTimeIndicatorType(inputRef.getType)) { + val resolvedRefType = input(inputRef.getIndex) + // input is a valid time indicator + if (isTimeIndicatorType(resolvedRefType)) { + inputRef + } + // input has been materialized + else { + new RexInputRef(inputRef.getIndex, resolvedRefType) + } + } + // reference is a regular field + else { + super.visitInputRef(inputRef) + } + } + + override def visitCall(call: RexCall): RexNode = { + val updatedCall = super.visitCall(call).asInstanceOf[RexCall] + + // materialize operands with time indicators + val materializedOperands = updatedCall.getOperator match { + + // skip materialization for special operators + case BasicOperatorTable.SESSION | BasicOperatorTable.HOP | BasicOperatorTable.TUMBLE => + updatedCall.getOperands.toList + + case _ => + updatedCall.getOperands.map { o => + if (isTimeIndicatorType(o.getType)) { + if (isRowtimeIndicatorType(o.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, o) + } else { + // generate proctime access + rexBuilder.makeCall(ProctimeSqlFunction, o) + } + } else { + o + } + } + } + + // remove time indicator return type + updatedCall.getOperator match { + + // we do not modify AS if operand has not been materialized + case SqlStdOperatorTable.AS if + isTimeIndicatorType(updatedCall.getOperands.get(0).getType) => + updatedCall + + // do not modify window time attributes + case BasicOperatorTable.TUMBLE_ROWTIME | + BasicOperatorTable.TUMBLE_PROCTIME | + BasicOperatorTable.HOP_ROWTIME | + BasicOperatorTable.HOP_PROCTIME | + BasicOperatorTable.SESSION_ROWTIME | + BasicOperatorTable.SESSION_PROCTIME + // since we materialize groupings on time indicators, + // we cannot check the operands anymore but the return type at least + if isTimeIndicatorType(updatedCall.getType) => + updatedCall + + // materialize function's result and operands + case _ if isTimeIndicatorType(updatedCall.getType) => + updatedCall.clone(timestamp, materializedOperands) + + // materialize function's operands only + case _ => + updatedCall.clone(updatedCall.getType, materializedOperands) + } + } +} diff --git a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala new file mode 100644 index 000000000..439e3a7f2 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -0,0 +1,1920 @@ +package org.apache.flink.table.codegen + +import java.math.{BigDecimal => JBigDecimal} + +import org.apache.calcite.avatica.util.DateTimeUtils +import org.apache.calcite.rex._ +import org.apache.calcite.sql.SqlOperator +import org.apache.calcite.sql.`type`.SqlTypeName._ +import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{ROW, _} +import org.apache.commons.lang3.StringEscapeUtils +import org.apache.flink.api.common.functions._ +import org.apache.flink.api.common.typeinfo._ +import org.apache.flink.api.common.typeutils.CompositeType +import org.apache.flink.api.java.typeutils._ +import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.{TableConfig, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.CodeGenUtils._ +import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE} +import org.apache.flink.table.codegen.calls.ScalarOperators._ +import org.apache.flink.table.codegen.calls.{CurrentTimePointCallGen, FunctionGenerator} +import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions, StreamRecordTimestampSqlFunction} +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils +import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo +import org.apache.flink.table.typeutils.TypeCheckUtils._ +import org.joda.time.format.DateTimeFormatter + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +/** + * [[CodeGenerator]] is the base code generator for generating Flink + * [[org.apache.flink.api.common.functions.Function]]s. + * It is responsible for expression generation and tracks the context (member variables etc). + * + * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. + * @param input1 type information about the first input of the Function + * @param input2 type information about the second input if the Function is binary + * @param input1FieldMapping additional mapping information for input1. + * POJO types have no deterministic field order and some input fields might not be read. + * The input1FieldMapping is also used to inject time indicator attributes. + * @param input2FieldMapping additional mapping information for input2. + * POJO types have no deterministic field order and some input fields might not be read. + */ +abstract class CodeGenerator( + config: TableConfig, + nullableInput: Boolean, + input1: TypeInformation[_ <: Any], + input2: Option[TypeInformation[_ <: Any]] = None, + input1FieldMapping: Option[Array[Int]] = None, + input2FieldMapping: Option[Array[Int]] = None) + extends RexVisitor[GeneratedExpression] { + + // check if nullCheck is enabled when inputs can be null + if (nullableInput && !config.getNullCheck) { + throw new CodeGenException("Null check must be enabled if entire rows can be null.") + } + + // check for POJO input1 mapping + input1 match { + case pt: PojoTypeInfo[_] => + input1FieldMapping.getOrElse( + throw new CodeGenException("No input mapping is specified for input1 of type POJO.")) + case _ => // ok + } + + // check for POJO input2 mapping + input2 match { + case Some(pt: PojoTypeInfo[_]) => + input2FieldMapping.getOrElse( + throw new CodeGenException("No input mapping is specified for input2 of type POJO.")) + case _ => // ok + } + + protected val input1Mapping: Array[Int] = input1FieldMapping match { + case Some(mapping) => mapping + case _ => (0 until input1.getArity).toArray + } + + protected val input2Mapping: Array[Int] = input2FieldMapping match { + case Some(mapping) => mapping + case _ => input2 match { + case Some(input) => (0 until input.getArity).toArray + case _ => Array[Int]() + } + } + + // set of member statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableMemberStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of constructor statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableInitStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of open statements for RichFunction that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableOpenStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of close statements for RichFunction that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableCloseStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of statements that will be added only once per record; + // code should only update member variables because local variables are not accessible if + // the code needs to be split; + // we use a LinkedHashSet to keep the insertion order + protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // map of initial input unboxing expressions that will be added only once + // (inputTerm, index) -> expr + protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = + mutable.Map[(String, Int), GeneratedExpression]() + + // set of constructor statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = + mutable.LinkedHashSet[(String, String)]() + + /** + * Flag that indicates that the generated code needed to be split into several methods. + */ + protected var hasCodeSplits: Boolean = false + + /** + * @return code block of statements that need to be placed in the member area of the Function + * (e.g. member variables and their initialization) + */ + def reuseMemberCode(): String = { + reusableMemberStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the constructor of the Function + */ + def reuseInitCode(): String = { + reusableInitStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the open() method of RichFunction + */ + def reuseOpenCode(): String = { + reusableOpenStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the close() method of RichFunction + */ + def reuseCloseCode(): String = { + reusableCloseStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the SAM of the Function + */ + def reusePerRecordCode(): String = { + reusablePerRecordStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that unbox input variables to a primitive variable + * and a corresponding null flag variable + */ + def reuseInputUnboxingCode(): String = { + reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n") + } + + /** + * @return code block of constructor statements for the Function + */ + def reuseConstructorCode(className: String): String = { + reusableConstructorStatements.map { case (params, body) => + s""" + |public $className($params) throws Exception { + | this(); + | $body + |} + |""".stripMargin + }.mkString("", "\n", "\n") + } + + /** + * @return term of the (casted and possibly boxed) first input + */ + var input1Term = "in1" + + /** + * @return term of the (casted and possibly boxed) second input + */ + var input2Term = "in2" + + /** + * @return term of the (casted) output collector + */ + var collectorTerm = "c" + + /** + * @return term of the output record (possibly defined in the member area e.g. Row, Tuple) + */ + var outRecordTerm = "out" + + /** + * @return term of the [[ProcessFunction]]'s context + */ + var contextTerm = "ctx" + + /** + * @return returns if null checking is enabled + */ + def nullCheck: Boolean = config.getNullCheck + + /** + * Generates an expression from a RexNode. If objects or variables can be reused, they will be + * added to reusable code sections internally. + * + * @param rex Calcite row expression + * @return instance of GeneratedExpression + */ + def generateExpression(rex: RexNode): GeneratedExpression = { + rex.accept(this) + } + + /** + * Generates an expression that converts the first input (and second input) into the given type. + * If two inputs are converted, the second input is appended. If objects or variables can + * be reused, they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param returnType conversion target type. Inputs and output must have the same arity. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @param rowtimeExpression an expression to extract the value of a rowtime field from + * the input data. Required if the field indices include a rowtime + * marker. + * @return instance of GeneratedExpression + */ + def generateConverterResultExpression( + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String], + rowtimeExpression: Option[RexNode] = None) + : GeneratedExpression = { + + val input1AccessExprs = input1Mapping.map { + case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | + TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER if rowtimeExpression.isDefined => + // generate rowtime attribute from expression + generateExpression(rowtimeExpression.get) + case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | + TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER => + throw TableException("Rowtime extraction expression missing. Please report a bug.") + case TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER => + // attribute is proctime indicator. + // we use a null literal and generate a timestamp when we need it. + generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) + case TimeIndicatorTypeInfo.PROCTIME_BATCH_MARKER => + // attribute is proctime field in a batch query. + // it is initialized with the current time. + generateCurrentTimestamp() + case idx => + generateInputAccess(input1, input1Term, idx) + } + + val input2AccessExprs = input2 match { + case Some(ti) => + input2Mapping.map(idx => generateInputAccess(ti, input2Term, idx)).toSeq + case None => Seq() // add nothing + } + + generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames) + } + + /** + * Generates an expression from the left input and the right table function. + */ + def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = { + val input1AccessExprs = input1Mapping.map { idx => + generateInputAccess(input1, input1Term, idx) + } + + val input2AccessExprs = input2 match { + case Some(ti) => + // use generateFieldAccess instead of generateInputAccess to avoid the generated table + // function's field access code is put on the top of function body rather than + // the while loop + input2Mapping.map { idx => + generateFieldAccess(ti, input2Term, idx) + }.toSeq + case None => throw new CodeGenException("Type information of input2 must not be null.") + } + (input1AccessExprs, input2AccessExprs) + } + + /** + * Generates an expression from a sequence of RexNode. If objects or variables can be reused, + * they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param returnType conversion target type. Type must have the same arity than rexNodes. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @param rexNodes sequence of RexNode to be converted + * @return instance of GeneratedExpression + */ + def generateResultExpression( + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String], + rexNodes: Seq[RexNode]) + : GeneratedExpression = { + val fieldExprs = rexNodes.map(generateExpression) + generateResultExpression(fieldExprs, returnType, resultFieldNames) + } + + /** + * Generates an expression from a sequence of other expressions. If objects or variables can + * be reused, they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param fieldExprs field expressions to be converted + * @param returnType conversion target type. Type must have the same arity than fieldExprs. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @return instance of GeneratedExpression + */ + def generateResultExpression( + fieldExprs: Seq[GeneratedExpression], + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String]) + : GeneratedExpression = { + // initial type check + if (returnType.getArity != fieldExprs.length) { + throw new CodeGenException( + s"Arity [${returnType.getArity}] of result type [$returnType] does not match " + + s"number [${fieldExprs.length}] of expressions [$fieldExprs].") + } + if (resultFieldNames.length != fieldExprs.length) { + throw new CodeGenException( + s"Arity [${resultFieldNames.length}] of result field names [$resultFieldNames] does not " + + s"match number [${fieldExprs.length}] of expressions [$fieldExprs].") + } + // type check + returnType match { + case pt: PojoTypeInfo[_] => + fieldExprs.zipWithIndex foreach { + case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression [$fieldExpr] type is" + + s" [${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") + + case _ => // ok + } + + case ct: CompositeType[_] => + fieldExprs.zipWithIndex foreach { + case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression[$fieldExpr] type is " + + s"[${fieldExpr.resultType}], result type is [${ct.getTypeAt(i)}]") + case _ => // ok + } + + case t: TypeInformation[_] if t != fieldExprs.head.resultType => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression [${fieldExprs.head}] " + + s"type is [${fieldExprs.head.resultType}], result type is [$t]") + + case _ => // ok + } + + val returnTypeTerm = boxedTypeTermForTypeInfo(returnType) + val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing) + + // generate result expression + returnType match { + case ri: RowTypeInfo => + addReusableOutRecord(ri) + val resultSetters = boxedFieldExprs.zipWithIndex map { + case (fieldExpr, i) => + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | $outRecordTerm.setField($i, null); + |} + |else { + | $outRecordTerm.setField($i, ${fieldExpr.resultTerm}); + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.setField($i, ${fieldExpr.resultTerm}); + |""".stripMargin + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case pt: PojoTypeInfo[_] => + addReusableOutRecord(pt) + val resultSetters = boxedFieldExprs.zip(resultFieldNames) map { + case (fieldExpr, fieldName) => + val accessor = getFieldAccessor(pt.getTypeClass, fieldName) + + accessor match { + // Reflective access of primitives/Objects + case ObjectPrivateFieldAccessor(field) => + val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName) + + val defaultIfNull = if (isFieldPrimitive(field)) { + primitiveDefaultValue(fieldExpr.resultType) + } else { + "null" + } + + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | ${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + defaultIfNull)}; + |} + |else { + | ${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + fieldExpr.resultTerm)}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + fieldExpr.resultTerm)}; + |""".stripMargin + } + + // primitive or Object field access (implicit boxing) + case _ => + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | $outRecordTerm.$fieldName = null; + |} + |else { + | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |""".stripMargin + } + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case tup: TupleTypeInfo[_] => + addReusableOutRecord(tup) + val resultSetters = boxedFieldExprs.zipWithIndex map { + case (fieldExpr, i) => + val fieldName = "f" + i + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be stored in a Tuple."); + |} + |else { + | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |""".stripMargin + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case _: CaseClassTypeInfo[_] => + val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n") + val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ") + val resultTerm = newName(outRecordTerm) + + val nullCheckCode = if (nullCheck) { + boxedFieldExprs map { (fieldExpr) => + s""" + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be stored in a Case Class."); + |} + |""".stripMargin + } mkString "\n" + } else { + "" + } + + val resultCode = + s""" + |$fieldCodes + |$nullCheckCode + |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams); + |""".stripMargin + + // case classes are not splittable + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, returnType) + + case _: TypeInformation[_] => + val fieldExpr = boxedFieldExprs.head + val nullCheckCode = if (nullCheck) { + s""" + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be used for atomic types."); + |} + |""".stripMargin + } else { + "" + } + val resultCode = + s""" + |${fieldExpr.code} + |$nullCheckCode + |""".stripMargin + + // other types are not splittable + GeneratedExpression(fieldExpr.resultTerm, fieldExpr.nullTerm, resultCode, returnType) + + case _ => + throw new CodeGenException(s"Unsupported result type: $returnType") + } + } + + // ---------------------------------------------------------------------------------------------- + // RexVisitor methods + // ---------------------------------------------------------------------------------------------- + + override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = { + // if inputRef index is within size of input1 we work with input1, input2 otherwise + val input = if (inputRef.getIndex < input1.getArity) { + (input1, input1Term) + } else { + (input2.getOrElse(throw new CodeGenException("Invalid input access.")), input2Term) + } + + val index = if (input._2 == input1Term) { + inputRef.getIndex + } else { + inputRef.getIndex - input1.getArity + } + + generateInputAccess(input._1, input._2, index) + } + + override def visitTableInputRef(rexTableInputRef: RexTableInputRef): GeneratedExpression = + visitInputRef(rexTableInputRef) + + override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = { + val refExpr = rexFieldAccess.getReferenceExpr.accept(this) + val index = rexFieldAccess.getField.getIndex + generateFieldAccess(refExpr, index) + } + + override def visitLiteral(literal: RexLiteral): GeneratedExpression = { + val resultType = FlinkTypeFactory.toTypeInfo(literal.getType) + val value = literal.getValue3 + // null value with type + if (value == null) { + return generateNullLiteral(resultType) + } + // non-null values + literal.getType.getSqlTypeName match { + + case BOOLEAN => + generateNonNullLiteral(resultType, literal.getValue3.toString) + + case TINYINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidByte) { + generateNonNullLiteral(resultType, decimal.byteValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to byte.") + } + + case SMALLINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidShort) { + generateNonNullLiteral(resultType, decimal.shortValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to short.") + } + + case INTEGER => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidInt) { + generateNonNullLiteral(resultType, decimal.intValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to integer.") + } + + case BIGINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidLong) { + generateNonNullLiteral(resultType, decimal.longValue().toString + "L") + } + else { + throw new CodeGenException("Decimal can not be converted to long.") + } + + case FLOAT => + val floatValue = value.asInstanceOf[JBigDecimal].floatValue() + floatValue match { + case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN") + case Float.NegativeInfinity => + generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY") + case Float.PositiveInfinity => + generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY") + case _ => generateNonNullLiteral(resultType, floatValue.toString + "f") + } + + case DOUBLE => + val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue() + doubleValue match { + case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN") + case Double.NegativeInfinity => + generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY") + case Double.PositiveInfinity => + generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY") + case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d") + } + case DECIMAL => + val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal]) + generateNonNullLiteral(resultType, decimalField) + + case VARCHAR | CHAR => + val escapedValue = StringEscapeUtils.escapeJava( + StringEscapeUtils.unescapeJava(value.toString) + ) + generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") + + case SYMBOL => + generateSymbol(value.asInstanceOf[Enum[_]]) + + case DATE => + generateNonNullLiteral(resultType, value.toString) + + case TIME => + generateNonNullLiteral(resultType, value.toString) + + case TIMESTAMP => + generateNonNullLiteral(resultType, value.toString + "L") + + case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidInt) { + generateNonNullLiteral(resultType, decimal.intValue().toString) + } else { + throw new CodeGenException( + s"Decimal '$decimal' can not be converted to interval of months.") + } + + case typeName if DAY_INTERVAL_TYPES.contains(typeName) => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidLong) { + generateNonNullLiteral(resultType, decimal.longValue().toString + "L") + } else { + throw new CodeGenException( + s"Decimal '$decimal' can not be converted to interval of milliseconds.") + } + + case t@_ => + throw new CodeGenException(s"Type not supported: $t") + } + } + + override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { + GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1) + } + + override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression = + throw new CodeGenException("Local variables are not supported yet.") + + override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression = + throw new CodeGenException("Range references are not supported yet.") + + override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression = + throw new CodeGenException("Dynamic parameter references are not supported yet.") + + override def visitCall(call: RexCall): GeneratedExpression = { + + // special case: time materialization + if (call.getOperator == ProctimeSqlFunction) { + return generateProctimeTimestamp() + } + + val resultType = FlinkTypeFactory.toTypeInfo(call.getType) + + // convert operands and help giving untyped NULL literals a type + val operands = call.getOperands.zipWithIndex.map { + + // this helps e.g. for AS(null) + // we might need to extend this logic in case some rules do not create typed NULLs + case (operandLiteral: RexLiteral, 0) if + operandLiteral.getType.getSqlTypeName == SqlTypeName.NULL && + call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => + generateNullLiteral(resultType) + + case (o@_, _) => + o.accept(this) + } + generateCall(call, operands, resultType) + } + def generateCall( + call: RexCall, + operands: Seq[GeneratedExpression], + resultType: TypeInformation[_]) + : GeneratedExpression = { + + call.getOperator match { + // arithmetic + case PLUS if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) + + case PLUS | DATETIME_PLUS if isTemporal(resultType) => + val left = operands.head + val right = operands(1) + requireTemporal(left) + requireTemporal(right) + generateTemporalPlusMinus(plus = true, nullCheck, left, right, config) + + case MINUS if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("-", nullCheck, resultType, left, right, config) + + case MINUS | MINUS_DATE if isTemporal(resultType) => + val left = operands.head + val right = operands(1) + requireTemporal(left) + requireTemporal(right) + generateTemporalPlusMinus(plus = false, nullCheck, left, right, config) + + case MULTIPLY if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) + + case MULTIPLY if isTimeInterval(resultType) => + val left = operands.head + val right = operands(1) + requireTimeInterval(left) + requireNumeric(right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) + + case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("/", nullCheck, resultType, left, right, config) + + case MOD if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("%", nullCheck, resultType, left, right, config) + + case UNARY_MINUS if isNumeric(resultType) => + val operand = operands.head + requireNumeric(operand) + generateUnaryArithmeticOperator("-", nullCheck, resultType, operand) + + case UNARY_MINUS if isTimeInterval(resultType) => + val operand = operands.head + requireTimeInterval(operand) + generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand) + + case UNARY_PLUS if isNumeric(resultType) => + val operand = operands.head + requireNumeric(operand) + generateUnaryArithmeticOperator("+", nullCheck, resultType, operand) + + case UNARY_PLUS if isTimeInterval(resultType) => + val operand = operands.head + requireTimeInterval(operand) + generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand) + + // comparison + case EQUALS => + val left = operands.head + val right = operands(1) + generateEquals(nullCheck, left, right) + + case NOT_EQUALS => + val left = operands.head + val right = operands(1) + generateNotEquals(nullCheck, left, right) + + case GREATER_THAN => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison(">", nullCheck, left, right) + + case GREATER_THAN_OR_EQUAL => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison(">=", nullCheck, left, right) + + case LESS_THAN => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison("<", nullCheck, left, right) + + case LESS_THAN_OR_EQUAL => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison("<=", nullCheck, left, right) + + case IS_NULL => + val operand = operands.head + generateIsNull(nullCheck, operand) + + case IS_NOT_NULL => + val operand = operands.head + generateIsNotNull(nullCheck, operand) + + // logic + case AND => + operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => + requireBoolean(left) + requireBoolean(right) + generateAnd(nullCheck, left, right) + } + + case OR => + operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => + requireBoolean(left) + requireBoolean(right) + generateOr(nullCheck, left, right) + } + + case NOT => + val operand = operands.head + requireBoolean(operand) + generateNot(nullCheck, operand) + + case CASE => + generateIfElse(nullCheck, operands, resultType) + + case IS_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsTrue(operand) + + case IS_NOT_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsNotTrue(operand) + + case IS_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsFalse(operand) + + case IS_NOT_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsNotFalse(operand) + + case IN => + val left = operands.head + val right = operands.tail + generateIn(this, left, right) + + // casting + case CAST | REINTERPRET => + val operand = operands.head + generateCast(nullCheck, operand, resultType) + + // as / renaming + case AS => + operands.head + + // string arithmetic + case CONCAT => + val left = operands.head + val right = operands(1) + requireString(left) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) + + // rows + case ROW => + generateRow(this, resultType, operands) + + // arrays + case ARRAY_VALUE_CONSTRUCTOR => + generateArray(this, resultType, operands) + + // maps + case MAP_VALUE_CONSTRUCTOR => + generateMap(this, resultType, operands) + + case ITEM => + operands.head.resultType match { + case t: TypeInformation[_] if isArray(t) => + val array = operands.head + val index = operands(1) + requireInteger(index) + generateArrayElementAt(this, array, index) + + case t: TypeInformation[_] if isMap(t) => + val key = operands(1) + generateMapGet(this, operands.head, key) + + case _ => throw new CodeGenException("Expect an array or a map.") + } + + case CARDINALITY => + operands.head.resultType match { + case t: TypeInformation[_] if isArray(t) => + val array = operands.head + generateArrayCardinality(nullCheck, array) + + case t: TypeInformation[_] if isMap(t) => + val map = operands.head + generateMapCardinality(nullCheck, map) + + case _ => throw new CodeGenException("Expect an array or a map.") + } + + case ELEMENT => + val array = operands.head + requireArray(array) + generateArrayElement(this, array) + + case DOT => + // Due to https://issues.apache.org/jira/browse/CALCITE-2162, expression such as + // "array[1].a.b" won't work now. + if (operands.size > 2) { + throw new CodeGenException( + "A DOT operator with more than 2 operands is not supported yet.") + } + val fieldName = call.operands.get(1).asInstanceOf[RexLiteral].getValueAs(classOf[String]) + val fieldIdx = operands + .head + .resultType + .asInstanceOf[CompositeType[_]] + .getFieldIndex(fieldName) + generateFieldAccess(operands.head, fieldIdx) + + case ScalarSqlFunctions.CONCAT => + generateConcat(this.nullCheck, operands) + + case ScalarSqlFunctions.CONCAT_WS => + generateConcatWs(operands) + + case StreamRecordTimestampSqlFunction => + generateStreamRecordRowtimeAccess() + + // advanced scalar functions + case sqlOperator: SqlOperator => + val callGen = FunctionGenerator.getCallGenerator( + sqlOperator, + operands.map(_.resultType), + resultType) + callGen + .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" + + s"If you think this function should be supported, " + + s"you can create an issue and start a discussion for it.")) + .generate(this, operands) + + // unknown or invalid + case call@_ => + throw new CodeGenException(s"Unsupported call: $call") + } + } + + override def visitOver(over: RexOver): GeneratedExpression = + throw new CodeGenException("Aggregate functions over windows are not supported yet.") + + override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression = + throw new CodeGenException("Subqueries are not supported yet.") + + override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): GeneratedExpression = + throw new CodeGenException("Pattern field references are not supported yet.") + + // ---------------------------------------------------------------------------------------------- + // generator helping methods + // ---------------------------------------------------------------------------------------------- + + private def generateCodeSplits(splits: Seq[String]): String = { + val totalLen = splits.map(_.length + 1).sum // 1 for a line break + + // split + if (totalLen > config.getMaxGeneratedCodeLength) { + + hasCodeSplits = true + + // add input unboxing to member area such that all split functions can access it + reusableInputUnboxingExprs.foreach { case (_, expr) => + + // declaration + val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) + if (nullCheck) { + reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") + } + reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") + + // assignment + if (nullCheck) { + reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") + } + reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") + } + + // add split methods to the member area and return the code necessary to call those methods + val methodCalls = splits.map { split => + val methodName = newName(s"split") + + val method = + s""" + |private final void $methodName() throws Exception { + | $split + |} + |""".stripMargin + reusableMemberStatements.add(method) + + // create method call + s"$methodName();" + } + + methodCalls.mkString("\n") + } + // don't split + else { + splits.mkString("\n") + } + } + + def generateFieldAccess(refExpr: GeneratedExpression, index: Int): GeneratedExpression = { + + val fieldAccessExpr = generateFieldAccess( + refExpr.resultType, + refExpr.resultTerm, + index) + + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType) + val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType) + val resultCode = if (nullCheck) { + s""" + |${refExpr.code} + |$resultTypeTerm $resultTerm; + |boolean $nullTerm; + |if (${refExpr.nullTerm}) { + | $resultTerm = $defaultValue; + | $nullTerm = true; + |} + |else { + | ${fieldAccessExpr.code} + | $resultTerm = ${fieldAccessExpr.resultTerm}; + | $nullTerm = ${fieldAccessExpr.nullTerm}; + |} + |""".stripMargin + } else { + s""" + |${refExpr.code} + |${fieldAccessExpr.code} + |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm}; + |""".stripMargin + } + + GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType) + } + + private def generateInputAccess( + inputType: TypeInformation[_ <: Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + // if input has been used before, we can reuse the code that + // has already been generated + val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match { + // input access and unboxing has already been generated + case Some(expr) => + expr + + // generate input access and unboxing if necessary + case None => + val expr = if (nullableInput) { + generateNullableInputFieldAccess(inputType, inputTerm, index) + } else { + generateFieldAccess(inputType, inputTerm, index) + } + + reusableInputUnboxingExprs((inputTerm, index)) = expr + expr + } + // hide the generated code as it will be executed only once + GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) + } + + private def generateNullableInputFieldAccess( + inputType: TypeInformation[_ <: Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val fieldType = inputType match { + case ct: CompositeType[_] => ct.getTypeAt(index) + case t: TypeInformation[_] => t + } + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + val defaultValue = primitiveDefaultValue(fieldType) + val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index) + + val inputCheckCode = + s""" + |$resultTypeTerm $resultTerm; + |boolean $nullTerm; + |if ($inputTerm == null) { + | $resultTerm = $defaultValue; + | $nullTerm = true; + |} + |else { + | ${fieldAccessExpr.code} + | $resultTerm = ${fieldAccessExpr.resultTerm}; + | $nullTerm = ${fieldAccessExpr.nullTerm}; + |} + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType) + } + + def generateFieldAccess( + inputType: TypeInformation[_], + inputTerm: String, + index: Int) + : GeneratedExpression = { + inputType match { + + case ct: CompositeType[_] => + val accessor = fieldAccessorFor(ct, index) + val fieldType: TypeInformation[Any] = ct.getTypeAt(index) + val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) + + accessor match { + case ObjectFieldAccessor(field) => + // primitive + if (isFieldPrimitive(field)) { + generateTerm(fieldType, s"$inputTerm.${field.getName}") + } + // Object + else { + generateInputFieldUnboxing( + fieldType, + s"($fieldTypeTerm) $inputTerm.${field.getName}") + } + + case ObjectGenericFieldAccessor(fieldName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" + generateInputFieldUnboxing(fieldType, inputCode) + + case ObjectMethodAccessor(methodName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" + generateInputFieldUnboxing(fieldType, inputCode) + + case ProductAccessor(i) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)" + generateInputFieldUnboxing(fieldType, inputCode) + + case ObjectPrivateFieldAccessor(field) => + val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) + val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) + // primitive + if (isFieldPrimitive(field)) { + generateTerm(fieldType, reflectiveAccessCode) + } + // Object + else { + generateInputFieldUnboxing(fieldType, reflectiveAccessCode) + } + } + + case t: TypeInformation[_] => + val fieldTypeTerm = boxedTypeTermForTypeInfo(t) + val inputCode = s"($fieldTypeTerm) $inputTerm" + generateInputFieldUnboxing(t, inputCode) + } + } + + private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { + val resultTerm = newName("result") + val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + if (nullCheck) { + val wrappedCode = s""" + |$resultTypeTerm $resultTerm = $defaultValue; + |""".stripMargin + + // mark this expression as a constant literal + GeneratedExpression(resultTerm, ALWAYS_NULL, wrappedCode, resultType, literal = true) + } else { + throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.") + } + } + + private[flink] def generateNonNullLiteral( + literalType: TypeInformation[_], + literalCode: String) + : GeneratedExpression = { + + // mark this expression as a constant literal + generateTerm(literalType, literalCode).copy(literal = true) + } + + private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = { + GeneratedExpression( + qualifyEnum(enum), + NEVER_NULL, + NO_CODE, + new GenericTypeInfo(enum.getDeclaringClass)) + } + + /** + * Generates access to a term (e.g. a field) that does not require unboxing logic. + * + * @param fieldType type of field + * @param fieldTerm expression term of field (already unboxed) + * @return internal unboxed field representation + */ + private[flink] def generateTerm( + fieldType: TypeInformation[_], + fieldTerm: String) + : GeneratedExpression = { + val resultTerm = newName("result") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + + val resultCode = s""" + |$resultTypeTerm $resultTerm = $fieldTerm; + |""".stripMargin + + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, fieldType) + } + + /** + * Converts the external boxed format to an internal mostly primitive field representation. + * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External + * objects are converted to their internal representation (Timestamp -> internal timestamp + * in long). + * + * @param fieldType type of field + * @param fieldTerm expression term of field to be unboxed + * @return internal unboxed field representation + */ + private[flink] def generateInputFieldUnboxing( + fieldType: TypeInformation[_], + fieldTerm: String) + : GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + val defaultValue = primitiveDefaultValue(fieldType) + + // explicit unboxing + val unboxedFieldCode = if (isTimePoint(fieldType)) { + timePointToInternalCode(fieldType, fieldTerm) + } else { + fieldTerm + } + + val wrappedCode = if (nullCheck && !isReference(fieldType)) { + // assumes that fieldType is a boxed primitive. + s""" + |boolean $nullTerm = $fieldTerm == null; + |$resultTypeTerm $resultTerm; + |if ($nullTerm) { + | $resultTerm = $defaultValue; + |} + |else { + | $resultTerm = $fieldTerm; + |} + |""".stripMargin + } else if (nullCheck) { + s""" + |boolean $nullTerm = $fieldTerm == null; + |$resultTypeTerm $resultTerm; + |if ($nullTerm) { + | $resultTerm = $defaultValue; + |} + |else { + | $resultTerm = ($resultTypeTerm) $unboxedFieldCode; + |} + |""".stripMargin + } else { + s""" + |$resultTypeTerm $resultTerm = ($resultTypeTerm) $unboxedFieldCode; + |""".stripMargin + } + + GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType) + } + + /** + * Converts the internal mostly primitive field representation to an external boxed format. + * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal + * representations are converted to their external objects (internal timestamp + * in long -> Timestamp). + * + * @param expr expression to be boxed + * @return external boxed field representation + */ + private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = { + expr.resultType match { + // convert internal date/time/timestamp to java.sql.* objects + case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP => + val resultTerm = newName("result") + val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType) + val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm) + + val resultCode = if (nullCheck) { + s""" + |${expr.code} + |$resultTypeTerm $resultTerm; + |if (${expr.nullTerm}) { + | $resultTerm = null; + |} + |else { + | $resultTerm = $convMethod; + |} + |""".stripMargin + } else { + s""" + |${expr.code} + |$resultTypeTerm $resultTerm = $convMethod; + |""".stripMargin + } + + GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType) + + // other types are autoboxed or need no boxing + case _ => expr + } + } + + private[flink] def generateNullableOutputBoxing( + expr: GeneratedExpression, + typeInfo: TypeInformation[_]) + : GeneratedExpression = { + val boxedExpr = generateOutputFieldBoxing(generateCast(nullCheck, expr, typeInfo)) + val boxedTypeTerm = boxedTypeTermForTypeInfo(typeInfo) + val exprOrNull: String = if (nullCheck) { + s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}" + } else { + boxedExpr.resultTerm + } + boxedExpr.copy(resultTerm = exprOrNull) + } + + private[flink] def generateStreamRecordRowtimeAccess(): GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val accessCode = + s""" + |Long $resultTerm = $contextTerm.timestamp(); + |if ($resultTerm == null) { + | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " + + | "TimestampAssigner is defined and the stream environment uses the EventTime time " + + | "characteristic."); + |} + |boolean $nullTerm = false; + """.stripMargin + + GeneratedExpression(resultTerm, nullTerm, accessCode, Types.LONG) + } + + private[flink] def generateProctimeTimestamp(): GeneratedExpression = { + val resultTerm = newName("result") + + val resultCode = + s""" + |long $resultTerm = $contextTerm.timerService().currentProcessingTime(); + |""".stripMargin + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) + } + + private[flink] def generateCurrentTimestamp(): GeneratedExpression = { + new CurrentTimePointCallGen(Types.SQL_TIMESTAMP, false).generate(this, Seq()) + } + + // ---------------------------------------------------------------------------------------------- + // Reusable code snippets + // ---------------------------------------------------------------------------------------------- + + /** + * Adds a reusable output record to the member area of the generated [[Function]]. + * The passed [[TypeInformation]] defines the type class to be instantiated. + * + * @param ti type information of type class to be instantiated during runtime + * @return member variable term + */ + def addReusableOutRecord(ti: TypeInformation[_]): Unit = { + val statement = ti match { + case rt: RowTypeInfo => + s""" + |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = + | new ${ti.getTypeClass.getCanonicalName}(${rt.getArity}); + |""".stripMargin + case _ => + s""" + |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = + | new ${ti.getTypeClass.getCanonicalName}(); + |""".stripMargin + } + reusableMemberStatements.add(statement) + } + + /** + * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]]. + * The field can be used for accessing POJO fields more efficiently during runtime, however, + * the field does not have to be public. + * + * @param clazz class of containing field + * @param fieldName name of field to be extracted and instantiated during runtime + * @return member variable term + */ + def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = { + val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName" + val fieldExtraction = + s""" + |final java.lang.reflect.Field $fieldTerm = + | org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField( + | ${clazz.getCanonicalName}.class, "$fieldName"); + |""".stripMargin + reusableMemberStatements.add(fieldExtraction) + + val fieldAccessibility = + s""" + |$fieldTerm.setAccessible(true); + |""".stripMargin + reusableInitStatements.add(fieldAccessibility) + + fieldTerm + } + + /** + * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]]. + * + * @param decimal decimal object to be instantiated during runtime + * @return member variable term + */ + def addReusableDecimal(decimal: JBigDecimal): String = decimal match { + case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO" + case JBigDecimal.ONE => "java.math.BigDecimal.ONE" + case JBigDecimal.TEN => "java.math.BigDecimal.TEN" + case _ => + val fieldTerm = newName("decimal") + val fieldDecimal = + s""" + |final java.math.BigDecimal $fieldTerm = + | new java.math.BigDecimal("${decimal.toString}"); + |""".stripMargin + reusableMemberStatements.add(fieldDecimal) + fieldTerm + } + + /** + * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]]. + * + * The seed parameter must be a literal/constant expression. + * + * @return member variable term + */ + def addReusableRandom(seedExpr: Option[GeneratedExpression]): String = { + val fieldTerm = newName("random") + + val field = + s""" + |final java.util.Random $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = seedExpr match { + case Some(s) if nullCheck => + s""" + |${s.code} + |if(!${s.nullTerm}) { + | $fieldTerm = new java.util.Random(${s.resultTerm}); + |} + |else { + | $fieldTerm = new java.util.Random(); + |} + |""".stripMargin + case Some(s) => + s""" + |${s.code} + |$fieldTerm = new java.util.Random(${s.resultTerm}); + |""".stripMargin + case _ => + s""" + |$fieldTerm = new java.util.Random(); + |""".stripMargin + } + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + /** + * Adds a reusable DateFormatter to the member area of the generated [[Function]]. + * + * @return member variable term + */ + def addReusableDateFormatter(format: GeneratedExpression): String = { + val fieldTerm = newName("dateFormatter") + + val field = + s""" + |final ${classOf[DateTimeFormatter].getCanonicalName} $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = + s""" + |${format.code} + |$fieldTerm = org.apache.flink.table.runtime.functions. + |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm}); + |""".stripMargin + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + /** + * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. + * + * @param function [[UserDefinedFunction]] object to be instantiated during runtime + * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] + * @return member variable term + */ + def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { + val classQualifier = function.getClass.getCanonicalName + val functionSerializedData = UserDefinedFunctionUtils.serialize(function) + val fieldTerm = s"function_${function.functionIdentifier}" + + val fieldFunction = + s""" + |final $classQualifier $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(fieldFunction) + + val functionDeserialization = + s""" + |$fieldTerm = ($classQualifier) + |${UserDefinedFunctionUtils.getClass.getName.stripSuffix("$")} + |.deserialize("$functionSerializedData"); + """.stripMargin + + reusableInitStatements.add(functionDeserialization) + + val openFunction = if (contextTerm != null) { + s""" + |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); + """.stripMargin + } else { + s""" + |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); + """.stripMargin + } + reusableOpenStatements.add(openFunction) + + val closeFunction = + s""" + |$fieldTerm.close(); + """.stripMargin + reusableCloseStatements.add(closeFunction) + + fieldTerm + } + + /** + * Adds a reusable constructor statement with the given parameter types. + * + * @param parameterTypes The parameter types to construct the function + * @return member variable terms + */ + def addReusableConstructor(parameterTypes: Class[_]*): Array[String] = { + val parameters = mutable.ListBuffer[String]() + val fieldTerms = mutable.ListBuffer[String]() + val body = mutable.ListBuffer[String]() + + parameterTypes.zipWithIndex.foreach { case (t, index) => + val classQualifier = t.getCanonicalName + val fieldTerm = newName(s"instance_${classQualifier.replace('.', '$')}") + val field = s"final $classQualifier $fieldTerm;" + reusableMemberStatements.add(field) + fieldTerms += fieldTerm + parameters += s"$classQualifier arg$index" + body += s"$fieldTerm = arg$index;" + } + + reusableConstructorStatements.add((parameters.mkString(","), body.mkString("", "\n", "\n"))) + + fieldTerms.toArray + } + + /** + * Adds a reusable [[org.apache.flink.types.Row]] + * to the member area of the generated [[Function]]. + */ + def addReusableRow(arity: Int): String = { + val fieldTerm = newName("row") + val fieldRow = + s""" + |final org.apache.flink.types.Row $fieldTerm = + | new org.apache.flink.types.Row($arity); + |""".stripMargin + reusableMemberStatements.add(fieldRow) + fieldTerm + } + + /** + * Adds a reusable array to the member area of the generated [[Function]]. + */ + def addReusableArray(clazz: Class[_], size: Int): String = { + val fieldTerm = newName("array") + val classQualifier = clazz.getCanonicalName // works also for int[] etc. + val initArray = classQualifier.replaceFirst("\\[", s"[$size") + val fieldArray = + s""" + |final $classQualifier $fieldTerm = + | new $initArray; + |""".stripMargin + reusableMemberStatements.add(fieldArray) + fieldTerm + } + + /** + * Adds a reusable hash map to the member area of the generated [[Function]]. + */ + def addReusableMap(): String = { + val fieldTerm = newName("map") + val classQualifier = "java.util.Map" + val initMap = "java.util.HashMap()" + val fieldMap = + s""" + |final $classQualifier $fieldTerm = + | new $initMap; + |""".stripMargin + reusableMemberStatements.add(fieldMap) + fieldTerm + } + + /** + * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTimestamp(): String = { + val fieldTerm = s"timestamp" + + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment + val field = + s""" + |$fieldTerm = java.lang.System.currentTimeMillis(); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTimestamp(): String = { + val fieldTerm = s"localtimestamp" + + val timestamp = addReusableTimestamp() + + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment + val field = + s""" + |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTime(): String = { + val fieldTerm = s"time" + + val timestamp = addReusableTimestamp() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() + val field = + s""" + |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |if (time < 0) { + | time += ${DateTimeUtils.MILLIS_PER_DAY}; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTime(): String = { + val fieldTerm = s"localtime" + + val localtimestamp = addReusableLocalTimestamp() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() + val field = + s""" + |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + + /** + * Adds a reusable date to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableDate(): String = { + val fieldTerm = s"date" + + val timestamp = addReusableTimestamp() + val time = addReusableTime() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() + val field = + s""" + |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); + |if ($time < 0) { + | $fieldTerm -= 1; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable [[java.util.HashSet]] to the member area of the generated [[Function]]. + * + * @param elements elements to be added to the set (including null) + * @return member variable term + */ + def addReusableSet(elements: Seq[GeneratedExpression]): String = { + val fieldTerm = newName("set") + + val field = + s""" + |final java.util.Set $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val init = + s""" + |$fieldTerm = new java.util.HashSet(); + |""".stripMargin + reusableInitStatements.add(init) + + elements.foreach { element => + val content = + s""" + |${element.code} + |if (${element.nullTerm}) { + | $fieldTerm.add(null); + |} else { + | $fieldTerm.add(${element.resultTerm}); + |} + |""".stripMargin + + reusableInitStatements.add(content) + } + + fieldTerm + } + + /** + * Adds a reusable constant to the member area of the generated [[Function]]. + * + * @param constant constant expression + * @return member variable term + */ + def addReusableBoxedConstant(constant: GeneratedExpression): String = { + require(constant.literal, "Literal expected") + + val fieldTerm = newName("constant") + + val boxed = generateOutputFieldBoxing(constant) + val boxedType = boxedTypeTermForTypeInfo(boxed.resultType) + + val field = + s""" + |final $boxedType $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val init = + s""" + |${boxed.code} + |$fieldTerm = ${boxed.resultTerm}; + |""".stripMargin + reusableInitStatements.add(init) + + fieldTerm + } + + /** + * Adds a reusable MessageDigest to the member area of the generated [[Function]]. + * + * @return member variable term + */ + def addReusableMessageDigest(algorithm: String): String = { + val fieldTerm = newName("messageDigest") + + val field = + s""" + |final java.security.MessageDigest $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = + s""" + |try { + | $fieldTerm = java.security.MessageDigest.getInstance("$algorithm"); + |} catch (java.security.NoSuchAlgorithmException e) { + | throw new RuntimeException("Algorithm for '$algorithm' is not available.", e); + |} + |""".stripMargin + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + def addReusableInitStatement(initStatement: String): Unit = { + reusableInitStatements.add(initStatement) + } + + def addReusableMemberStatement(memberStatement: String): Unit = { + reusableMemberStatements.add(memberStatement) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala new file mode 100644 index 000000000..d08dd570f --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala @@ -0,0 +1,574 @@ +package org.apache.flink.table.codegen + +import java.math.{BigDecimal => JBigDecimal} +import java.util + +import org.apache.calcite.rel.RelCollation +import org.apache.calcite.rex._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{CLASSIFIER, FINAL, FIRST, LAST, MATCH_NUMBER, NEXT, PREV, RUNNING} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.Indenter.toISC +import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName, primitiveDefaultValue} +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * A code generator for generating CEP related functions. + * + * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. + * @param input type information about the first input of the Function + * @param patternNames the names of patterns + * @param generateCondition whether the code generator is generating [[IterativeCondition]] + * @param patternName the name of current pattern + */ +class MatchCodeGenerator( + config: TableConfig, + nullableInput: Boolean, + input: TypeInformation[_ <: Any], + patternNames: Seq[String], + generateCondition: Boolean, + patternName: Option[String] = None) + extends CodeGenerator(config,nullableInput,input){ + /** + * @return term of pattern names + */ + private val patternNameListTerm = newName("patternNameList") + + /** + * @return term of current pattern which is processing + */ + private val currPatternTerm = newName("currPattern") + + /** + * @return term of current event which is processing + */ + private val currEventTerm = newName("currEvent") + + private val buildPatternNameList: String = { + for (patternName <- patternNames) yield + s""" + |$patternNameListTerm.add("$patternName"); + |""".stripMargin + }.mkString("\n") + + def addReusableStatements(): Unit = { + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val memberStatement = + s""" + |$eventTypeTerm $currEventTerm = null; + |String $currPatternTerm = null; + |java.util.List $patternNameListTerm = new java.util.ArrayList(); + |""".stripMargin + addReusableMemberStatement(memberStatement) + + addReusableInitStatement(buildPatternNameList) + } + + /** + * Generates a [[IterativeCondition]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedIterativeCondition + */ + def generateIterativeCondition( + name: String, + bodyCode: String) + : GeneratedIterativeCondition = { + + val funcName = newName(name) + val inputTypeTerm = boxedTypeTermForTypeInfo(input) + + val funcCode = j""" + public class $funcName + extends ${classOf[IterativeCondition[_]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public boolean filter( + Object _in1, ${classOf[IterativeCondition.Context[_]].getCanonicalName} $contextTerm) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedIterativeCondition(funcName, funcCode) + } + + /** + * Generates a [[PatternSelectFunction]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedPatternSelectFunction + */ + def generatePatternSelectFunction( + name: String, + bodyCode: String) + : GeneratedPatternSelectFunction = { + + val funcName = newName(name) + val inputTypeTerm = + classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName + + val funcCode = j""" + public class $funcName + implements ${classOf[PatternSelectFunction[_, _]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public Object select(java.util.Map> _in1) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedPatternSelectFunction(funcName, funcCode) + } + + /** + * Generates a [[PatternFlatSelectFunction]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedPatternFlatSelectFunction + */ + def generatePatternFlatSelectFunction( + name: String, + bodyCode: String) + : GeneratedPatternFlatSelectFunction = { + + val funcName = newName(name) + val inputTypeTerm = + classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName + + val funcCode = j""" + public class $funcName + implements ${classOf[PatternFlatSelectFunction[_, _]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public void flatSelect(java.util.Map> _in1, + org.apache.flink.util.Collector $collectorTerm) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedPatternFlatSelectFunction(funcName, funcCode) + } + + def generateSelectOutputExpression( + partitionKeys: util.List[RexNode], + measures: util.Map[String, RexNode], + returnType: RowSchema) + : GeneratedExpression = { + + val eventNameTerm = newName("event") + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + + // For "ONE ROW PER MATCH", the output columns include: + // 1) the partition columns; + // 2) the columns defined in the measures clause. + val resultExprs = + partitionKeys.asScala.map { case inputRef: RexInputRef => + generateFieldAccess(input, eventNameTerm, inputRef.getIndex) + } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => + generateExpression(measures.get(fieldName)) + } + + val resultExpression = generateResultExpression( + resultExprs, + returnType.typeInfo, + returnType.fieldNames) + + val resultCode = + s""" + |$eventTypeTerm $eventNameTerm = null; + |if (${partitionKeys.size()} > 0) { + | for (java.util.Map.Entry entry : $input1Term.entrySet()) { + | java.util.List value = (java.util.List) entry.getValue(); + | if (value != null && value.size() > 0) { + | $eventNameTerm = ($eventTypeTerm) value.get(0); + | break; + | } + | } + |} + | + |${resultExpression.code} + |""".stripMargin + + resultExpression.copy(code = resultCode) + } + + def generateFlatSelectOutputExpression( + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + measures: util.Map[String, RexNode], + returnType: RowSchema) + : GeneratedExpression = { + + val patternNameTerm = newName("patternName") + val eventNameTerm = newName("event") + val eventNameListTerm = newName("eventList") + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val listTypeTerm = classOf[java.util.List[_]].getCanonicalName + + // For "ALL ROWS PER MATCH", the output columns include: + // 1) the partition columns; + // 2) the ordering columns; + // 3) the columns defined in the measures clause; + // 4) any remaining columns defined of the input. + val fieldsAccessed = mutable.Set[Int]() + val resultExprs = + partitionKeys.asScala.map { case inputRef: RexInputRef => + fieldsAccessed += inputRef.getIndex + generateFieldAccess(input, eventNameTerm, inputRef.getIndex) + } ++ orderKeys.getFieldCollations.asScala.map { fieldCollation => + fieldsAccessed += fieldCollation.getFieldIndex + generateFieldAccess(input, eventNameTerm, fieldCollation.getFieldIndex) + } ++ (0 until input.getArity).filterNot(fieldsAccessed.contains).map { idx => + generateFieldAccess(input, eventNameTerm, idx) + } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => + generateExpression(measures.get(fieldName)) + } + + val resultExpression = generateResultExpression( + resultExprs, + returnType.typeInfo, + returnType.fieldNames) + + val resultCode = + s""" + |for (String $patternNameTerm : $patternNameListTerm) { + | $currPatternTerm = $patternNameTerm; + | $listTypeTerm $eventNameListTerm = ($listTypeTerm) $input1Term.get($patternNameTerm); + | if ($eventNameListTerm != null) { + | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { + | $currEventTerm = $eventNameTerm; + | ${resultExpression.code} + | $collectorTerm.collect(${resultExpression.resultTerm}); + | } + | } + |} + |$currPatternTerm = null; + |$currEventTerm = null; + |""".stripMargin + + GeneratedExpression("", "false", resultCode, null) + } + + override def visitCall(call: RexCall): GeneratedExpression = { + val resultType = FlinkTypeFactory.toTypeInfo(call.getType) + call.getOperator match { + case PREV => + val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generatePrev( + call.operands.get(0), + count, + resultType) + + case NEXT | CLASSIFIER | MATCH_NUMBER => + throw new CodeGenException(s"Unsupported call: $call") + + case FIRST | LAST => + val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generateFirstLast( + call.operands.get(0), + count, + resultType, + running = true, + call.getOperator == FIRST) + + case RUNNING | FINAL => + generateRunningFinal( + call.operands.get(0), + resultType, + call.getOperator == RUNNING) + + case _ => super.visitCall(call) + } + } + + private def generatePrev( + rexNode: RexNode, + count: Int, + resultType: TypeInformation[_]) + : GeneratedExpression = { + rexNode match { + case patternFieldRef: RexPatternFieldRef => + if (count == 0 && patternFieldRef.getAlpha == patternName.get) { + // return current one + return visitInputRef(patternFieldRef) + } + + val listName = newName("patternEvents") + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val indexTerm = newName("eventIndex") + val visitedEventNumberTerm = newName("visitedEventNumber") + val eventTerm = newName("event") + val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + + val patternNamesToVisit = patternNames + .take(patternNames.indexOf(patternFieldRef.getAlpha) + 1) + .reverse + def findEventByPhysicalPosition: String = { + val init: String = + s""" + |java.util.List $listName = new java.util.ArrayList(); + |""".stripMargin + + val getResult: String = { + for (tmpPatternName <- patternNamesToVisit) yield + s""" + |for ($eventTypeTerm $eventTerm : $contextTerm + | .getEventsForPattern("$tmpPatternName")) { + | $listName.add($eventTerm); + |} + | + |$indexTerm = $listName.size() - ($count - $visitedEventNumberTerm); + |if ($indexTerm >= 0) { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) $listName.get($indexTerm)) + | .getField(${patternFieldRef.getIndex}); + | $nullTerm = false; + | break; + |} + | + |$visitedEventNumberTerm += $listName.size(); + |$listName.clear(); + |""".stripMargin + }.mkString("\n") + + s""" + |$init + |$getResult + |""".stripMargin + } + + val resultCode = + s""" + |int $visitedEventNumberTerm = 0; + |int $indexTerm; + |$resultTypeTerm $resultTerm = $defaultValue; + |boolean $nullTerm = true; + |do { + | $findEventByPhysicalPosition + |} while (false); + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generatePrev( + operand, + count, + FlinkTypeFactory.toTypeInfo(operand.getType)) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } + + private def generateFirstLast( + rexNode: RexNode, + count: Int, + resultType: TypeInformation[_], + running: Boolean, + first: Boolean) + : GeneratedExpression = { + rexNode match { + case patternFieldRef: RexPatternFieldRef => + + val eventNameTerm = newName("event") + val resultTerm = newName("result") + val listName = newName("patternEvents") + val nullTerm = newName("isNull") + val patternNameTerm = newName("patternName") + val eventNameListTerm = newName("eventNameList") + val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val listTypeTerm = classOf[java.util.List[_]].getCanonicalName + + def findEventByLogicalPosition: String = { + val init = + s""" + |java.util.List $listName = new java.util.ArrayList(); + |""".stripMargin + + val findEventsByPatterName = if (generateCondition) { + s""" + |for ($eventTypeTerm $eventNameTerm : $contextTerm + | .getEventsForPattern("${patternFieldRef.getAlpha}")) { + | $listName.add($eventNameTerm); + |} + |""".stripMargin + } else { + s""" + |for (String $patternNameTerm : $patternNameListTerm) { + | if ($patternNameTerm.equals("${patternFieldRef.getAlpha}") || + | ${patternFieldRef.getAlpha.equals("*")}) { + | boolean skipLoop = false; + | $listTypeTerm $eventNameListTerm = + | ($listTypeTerm) $input1Term.get($patternNameTerm); + | if ($eventNameListTerm != null) { + | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { + | $listName.add($eventNameTerm); + | if ($running && $eventNameTerm == $currEventTerm) { + | skipLoop = true; + | break; + | } + | } + | } + | + | if (skipLoop) { + | break; + | } + | } + | + | if ($running && $patternNameTerm.equals($currPatternTerm)) { + | break; + | } + |} + |""".stripMargin + } + + val getResult = + s""" + |if ($listName.size() > $count) { + | if ($first) { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) + | $listName.get($count)) + | .getField(${patternFieldRef.getIndex}); + | } else { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) + | $listName.get($listName.size() - $count - 1)) + | .getField(${patternFieldRef.getIndex}); + | } + | $nullTerm = false; + |} + |""".stripMargin + + s""" + |$init + |$findEventsByPatterName + |$getResult + |""".stripMargin + } + + val resultCode = + s""" + |$resultTypeTerm $resultTerm = $defaultValue; + |boolean $nullTerm = true; + |$findEventByLogicalPosition + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generateFirstLast( + operand, + count, + FlinkTypeFactory.toTypeInfo(operand.getType), + running, + first) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } + + /* + + + */ + private def generateRunningFinal( + rexNode: RexNode, + resultType: TypeInformation[_], + running: Boolean): GeneratedExpression + = { + rexNode match { + case _: RexPatternFieldRef => + generateFirstLast(rexNode, 0, resultType, running, first = false) + + case rexCall: RexCall if rexCall.getOperator == FIRST || rexCall.getOperator == LAST => + val countLiteral = rexCall.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generateFirstLast( + rexCall.operands.get(0), + count, + resultType, + running, + rexCall.getOperator == FIRST) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generateRunningFinal( + operand, + FlinkTypeFactory.toTypeInfo(operand.getType), + running) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala new file mode 100644 index 000000000..3551d80ed --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala @@ -0,0 +1,100 @@ +package org.apache.flink.table.codegen + +import org.apache.flink.api.common.functions +import org.apache.flink.api.common.functions.Function +import org.apache.flink.api.common.io.InputFormat +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * Describes a generated expression. + * + * @param resultTerm term to access the result of the expression + * @param nullTerm boolean term that indicates if expression is null + * @param code code necessary to produce resultTerm and nullTerm + * @param resultType type of the resultTerm + * @param literal flag to indicate a constant expression do not reference input and can thus + * be used in the member area (e.g. as constructor parameter of a reusable + * instance) + */ +case class GeneratedExpression( + resultTerm: String, + nullTerm: String, + code: String, + resultType: TypeInformation[_], + literal: Boolean = false) + +object GeneratedExpression { + val ALWAYS_NULL = "true" + val NEVER_NULL = "false" + val NO_CODE = "" +} + +/** + * Describes a generated [[functions.Function]] + * + * @param name class name of the generated Function. + * @param returnType the type information of the result type + * @param code code of the generated Function. + * @tparam F type of function + * @tparam T type of function + */ +case class GeneratedFunction[F <: Function, T <: Any]( + name: String, + returnType: TypeInformation[T], + code: String) + +/** + * Describes a generated aggregate helper function + * + * @param name class name of the generated Function. + * @param code code of the generated Function. + */ +case class GeneratedAggregationsFunction( + name: String, + code: String) + +/** + * Describes a generated [[InputFormat]]. + * + * @param name class name of the generated input function. + * @param returnType the type information of the result type + * @param code code of the generated Function. + * @tparam F type of function + * @tparam T type of function + */ +case class GeneratedInput[F <: InputFormat[_, _], T <: Any]( + name: String, + returnType: TypeInformation[T], + code: String) + +/** + * Describes a generated [[org.apache.flink.util.Collector]]. + * + * @param name class name of the generated Collector. + * @param code code of the generated Collector. + */ +case class GeneratedCollector(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.pattern.conditions.IterativeCondition]]. + * + * @param name class name of the generated IterativeCondition. + * @param code code of the generated IterativeCondition. + */ +case class GeneratedIterativeCondition(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.PatternSelectFunction]]. + * + * @param name class name of the generated PatternSelectFunction. + * @param code code of the generated PatternSelectFunction. + */ +case class GeneratedPatternSelectFunction(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.PatternFlatSelectFunction]]. + * + * @param name class name of the generated PatternFlatSelectFunction. + * @param code code of the generated PatternFlatSelectFunction. + */ +case class GeneratedPatternFlatSelectFunction(name: String, code: String) diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala new file mode 100644 index 000000000..10f60ba88 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala @@ -0,0 +1,309 @@ +package org.apache.flink.table.plan.nodes.datastream + +import java.math.{BigDecimal => JBigDecimal} +import java.util + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.SqlTypeName._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable._ +import org.apache.flink.cep.{CEP, PatternStream} +import org.apache.flink.cep.pattern.Pattern +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.windowing.time.Time +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.runtime.RowtimeProcessFunction +import org.apache.flink.table.runtime.cepmatch.{ConvertToRow, MatchUtil} +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer + +/** + * Flink RelNode which matches along with LogicalMatch. + */ +class DataStreamMatch( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode, + schema: RowSchema, + inputSchema: RowSchema) + extends SingleRel(cluster, traitSet, input) + with DataStreamRel { + + override def deriveRowType(): RelDataType = schema.relDataType + + override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { + new DataStreamMatch( + cluster, + traitSet, + inputs.get(0), + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval, + schema, + inputSchema) + } + + override def toString: String = { + s"Match(${ + if (!partitionKeys.isEmpty) { + s"PARTITION BY: ${partitionKeys.toArray.map(_.toString).mkString(", ")}, " + } else { + "" + } + }${ + if (!orderKeys.getFieldCollations.isEmpty) { + s"ORDER BY: ${orderKeys.getFieldCollations.asScala.map { + x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName + }.mkString(", ")}, " + } else { + "" + } + }${ + if (!measures.isEmpty) { + s"MEASURES: ${measures.asScala.map { + case (k, v) => s"${v.toString} AS $k" + }.mkString(", ")}, " + } else { + "" + } + }${ + if (allRows) { + s"ALL ROWS PER MATCH, " + } else { + s"ONE ROW PER MATCH, " + } + }${ + s"${after.toString}, " + }${ + s"PATTERN: (${pattern.toString})" + }${ + if (interval != null) { + s"WITHIN INTERVAL: $interval, " + } else { + s", " + } + }${ + if (!subsets.isEmpty) { + s"SUBSET: ${subsets.asScala.map { + case (k, v) => s"$k = (${v.toArray.mkString(", ")})" + }.mkString(", ")}, " + } else { + "" + } + }${ + s"DEFINE: ${patternDefinitions.asScala.map { + case (k, v) => s"$k AS ${v.toString}" + }.mkString(", ")}" + })" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + pw.input("input", getInput()) + .itemIf("partitionBy", + partitionKeys.toArray.map(_.toString).mkString(", "), + !partitionKeys.isEmpty) + .itemIf("orderBy", + orderKeys.getFieldCollations.asScala.map { + x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName + }.mkString(", "), + !orderKeys.getFieldCollations.isEmpty) + .itemIf("measures", + measures.asScala.map { case (k, v) => s"${v.toString} AS $k"}.mkString(", "), + !measures.isEmpty) + .item("allrows", allRows) + .item("after", after.toString) + .item("pattern", pattern.toString) + .itemIf("within interval", + if (interval != null) { + interval.toString + } else { + null + }, + interval != null) + .itemIf("subset", + subsets.asScala.map { case (k, v) => s"$k = (${v.toArray.mkString(", ")})"}.mkString(", "), + !subsets.isEmpty) + .item("define", + patternDefinitions.asScala.map { case (k, v) => s"$k AS ${v.toString}"}.mkString(", ")) + } + + override def translateToPlan( + tableEnv: StreamTableEnvironment, + queryConfig: StreamQueryConfig): DataStream[CRow] = { + + val config = tableEnv.config + val inputTypeInfo = inputSchema.typeInfo + + val crowInput: DataStream[CRow] = getInput + .asInstanceOf[DataStreamRel] + .translateToPlan(tableEnv, queryConfig) + + val rowtimeFields = inputSchema.relDataType + .getFieldList.asScala + .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) + + //主要是针对rowtime做处理,对存在rowtime的将其复制到StreamRecord的时间戳字段内 + val timestampedInput = if (rowtimeFields.nonEmpty) { + // copy the rowtime field into the StreamRecord timestamp field + val timeIdx = rowtimeFields.head.getIndex + + crowInput + .process(new RowtimeProcessFunction(timeIdx, CRowTypeInfo(inputTypeInfo))) + .setParallelism(crowInput.getParallelism) + .name(s"rowtime field: (${rowtimeFields.head})") + } else { + crowInput + } + + val inputDS: DataStream[Row] = timestampedInput + .map(new ConvertToRow) + .setParallelism(timestampedInput.getParallelism) + .name("ConvertToRow") + .returns(inputTypeInfo) + + def translatePattern( + rexNode: RexNode, + currentPattern: Pattern[Row, Row], + patternNames: ListBuffer[String]): Pattern[Row, Row] = rexNode match { + //Conditions + case literal: RexLiteral => + val patternName = literal.getValue3.toString + patternNames += patternName + val newPattern = next(currentPattern, patternName) + + val patternDefinition = patternDefinitions.get(patternName) + if (patternDefinition != null) { + val condition = MatchUtil.generateIterativeCondition( + config, + inputSchema, + patternName, + patternNames, + patternDefinition, + inputTypeInfo) + + newPattern.where(condition) + } else { + newPattern + } + + case call: RexCall => + + call.getOperator match { + case PATTERN_CONCAT => + val left = call.operands.get(0) + val right = call.operands.get(1) + translatePattern(right, + translatePattern(left, currentPattern, patternNames), + patternNames) + // Quantifiers + case PATTERN_QUANTIFIER => + val name = call.operands.get(0).asInstanceOf[RexLiteral] + val newPattern = translatePattern(name, currentPattern, patternNames) + + val startNum = call.operands.get(1).asInstanceOf[RexLiteral] + .getValue3.asInstanceOf[JBigDecimal].intValue() + val endNum = call.operands.get(2).asInstanceOf[RexLiteral] + .getValue3.asInstanceOf[JBigDecimal].intValue() + + if (startNum == 0 && endNum == -1) { // zero or more + newPattern.oneOrMore().optional().consecutive() + } else if (startNum == 1 && endNum == -1) { // one or more + newPattern.oneOrMore().consecutive() + } else if (startNum == 0 && endNum == 1) { // optional + newPattern.optional() + } else if (endNum != -1) { // times + newPattern.times(startNum, endNum).consecutive() + } else { // times or more + newPattern.timesOrMore(startNum).consecutive() + } + + case PATTERN_ALTER => + throw TableException("Currently, CEP doesn't support branching patterns.") + + case PATTERN_PERMUTE => + throw TableException("Currently, CEP doesn't support PERMUTE patterns.") + + case PATTERN_EXCLUDE => + throw TableException("Currently, CEP doesn't support '{-' '-}' patterns.") + } + + case _ => + throw TableException("") + } + + val patternNames: ListBuffer[String] = ListBuffer() + val cepPattern = translatePattern(pattern, null, patternNames) + if (interval != null) { + val intervalLiteral = interval.asInstanceOf[RexLiteral] + val intervalValue = interval.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long]) + val intervalMs: Long = intervalLiteral.getTypeName match { + case INTERVAL_YEAR | INTERVAL_YEAR_MONTH | INTERVAL_MONTH => + // convert from months to milliseconds, suppose 1 month = 30 days + intervalValue * 30L * 24 * 3600 * 1000 + case _ => intervalValue + } + + cepPattern.within(Time.milliseconds(intervalMs)) + } + + val patternStream: PatternStream[Row] = CEP.pattern[Row](inputDS, cepPattern) + + val outTypeInfo = CRowTypeInfo(schema.typeInfo) + if (allRows) { + val patternFlatSelectFunction = + MatchUtil.generatePatternFlatSelectFunction( + config, + schema, + patternNames, + partitionKeys, + orderKeys, + measures, + inputTypeInfo) + patternStream.flatSelect[CRow](patternFlatSelectFunction, outTypeInfo) + } else { + val patternSelectFunction = + MatchUtil.generatePatternSelectFunction( + config, + schema, + patternNames, + partitionKeys, + measures, + inputTypeInfo) + patternStream.select[CRow](patternSelectFunction, outTypeInfo) + } + } + + private def next(currentPattern: Pattern[Row, Row], patternName: String): Pattern[Row, Row] = { + if (currentPattern == null) { + Pattern.begin(patternName) + } else { + currentPattern.next(patternName) + } + } +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala new file mode 100644 index 000000000..693e4d314 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala @@ -0,0 +1,115 @@ +package org.apache.flink.table.plan.nodes.logical + +import java.util + +import org.apache.calcite.plan._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.Match +import org.apache.calcite.rel.logical.LogicalMatch +import org.apache.calcite.rel.{RelCollation, RelNode} +import org.apache.calcite.rex.RexNode +import org.apache.flink.table.plan.nodes.FlinkConventions + + +class FlinkLogicalMatch( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + rowType: RelDataType, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, _ <: util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode) + extends Match( + cluster, + traitSet, + input, + rowType, + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval) + with FlinkLogicalRel { + + override def copy( + input: RelNode, + rowType: RelDataType, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, _ <: util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode): Match = { + new FlinkLogicalMatch( + cluster, + traitSet, + input, + rowType, + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval) + } +} + +private class FlinkLogicalMatchConverter + extends ConverterRule( + classOf[LogicalMatch], + Convention.NONE, + FlinkConventions.LOGICAL, + "FlinkLogicalMatchConverter") { + + override def convert(rel: RelNode): RelNode = { + val logicalMatch = rel.asInstanceOf[LogicalMatch] + val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) + val newInput = RelOptRule.convert(logicalMatch.getInput, FlinkConventions.LOGICAL) + + new FlinkLogicalMatch( + rel.getCluster, + traitSet, + newInput, + logicalMatch.getRowType, + logicalMatch.getPattern, + logicalMatch.isStrictStart, + logicalMatch.isStrictEnd, + logicalMatch.getPatternDefinitions, + logicalMatch.getMeasures, + logicalMatch.getAfter, + logicalMatch.getSubsets, + logicalMatch.isAllRows, + logicalMatch.getPartitionKeys, + logicalMatch.getOrderKeys, + logicalMatch.getInterval) + } +} + +object FlinkLogicalMatch { + val CONVERTER: ConverterRule = new FlinkLogicalMatchConverter() +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala new file mode 100644 index 000000000..5a0ff9d65 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -0,0 +1,209 @@ +package org.apache.flink.table.plan.rules + +import org.apache.calcite.rel.core.RelFactories +import org.apache.calcite.rel.rules._ +import org.apache.calcite.tools.{RuleSet, RuleSets} +import org.apache.flink.table.plan.rules.common._ +import org.apache.flink.table.plan.rules.logical._ +import org.apache.flink.table.plan.rules.dataSet._ +import org.apache.flink.table.plan.rules.datastream.{DataStreamMatchRule, _} +import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalMatch, _} + +object FlinkRuleSets { + + /** + * Convert sub-queries before query decorrelation. + */ + val TABLE_SUBQUERY_RULES: RuleSet = RuleSets.ofList( + SubQueryRemoveRule.FILTER, + SubQueryRemoveRule.PROJECT, + SubQueryRemoveRule.JOIN) + + /** + * Convert table references before query decorrelation. + */ + val TABLE_REF_RULES: RuleSet = RuleSets.ofList( + TableScanRule.INSTANCE, + EnumerableToLogicalTableScan.INSTANCE) + + val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList( + + // push a filter into a join + FilterJoinRule.FILTER_ON_JOIN, + // push filter into the children of a join + FilterJoinRule.JOIN, + // push filter through an aggregation + FilterAggregateTransposeRule.INSTANCE, + // push filter through set operation + FilterSetOpTransposeRule.INSTANCE, + // push project through set operation + ProjectSetOpTransposeRule.INSTANCE, + + // aggregation and projection rules + AggregateProjectMergeRule.INSTANCE, + AggregateProjectPullUpConstantsRule.INSTANCE, + // push a projection past a filter or vice versa + ProjectFilterTransposeRule.INSTANCE, + FilterProjectTransposeRule.INSTANCE, + // push a projection to the children of a join + // push all expressions to handle the time indicator correctly + new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), + // merge projections + ProjectMergeRule.INSTANCE, + // remove identity project + ProjectRemoveRule.INSTANCE, + // reorder sort and projection + SortProjectTransposeRule.INSTANCE, + ProjectSortTransposeRule.INSTANCE, + + // join rules + JoinPushExpressionsRule.INSTANCE, + + // remove union with only a single child + UnionEliminatorRule.INSTANCE, + // convert non-all union into all-union + distinct + UnionToDistinctRule.INSTANCE, + + // remove aggregation if it does not aggregate and input is already distinct + AggregateRemoveRule.INSTANCE, + // push aggregate through join + AggregateJoinTransposeRule.EXTENDED, + // aggregate union rule + AggregateUnionAggregateRule.INSTANCE, + // expand distinct aggregate to normal aggregate with groupby + AggregateExpandDistinctAggregatesRule.JOIN, + + // reduce aggregate functions like AVG, STDDEV_POP etc. + AggregateReduceFunctionsRule.INSTANCE, + + // remove unnecessary sort rule + SortRemoveRule.INSTANCE, + + // prune empty results rules + PruneEmptyRules.AGGREGATE_INSTANCE, + PruneEmptyRules.FILTER_INSTANCE, + PruneEmptyRules.JOIN_LEFT_INSTANCE, + PruneEmptyRules.JOIN_RIGHT_INSTANCE, + PruneEmptyRules.PROJECT_INSTANCE, + PruneEmptyRules.SORT_INSTANCE, + PruneEmptyRules.UNION_INSTANCE, + + // calc rules + FilterCalcMergeRule.INSTANCE, + ProjectCalcMergeRule.INSTANCE, + FilterToCalcRule.INSTANCE, + ProjectToCalcRule.INSTANCE, + CalcMergeRule.INSTANCE, + + // scan optimization + PushProjectIntoTableSourceScanRule.INSTANCE, + PushFilterIntoTableSourceScanRule.INSTANCE, + + // unnest rule + LogicalUnnestRule.INSTANCE, + + // translate to flink logical rel nodes + FlinkLogicalAggregate.CONVERTER, + FlinkLogicalWindowAggregate.CONVERTER, + FlinkLogicalOverWindow.CONVERTER, + FlinkLogicalCalc.CONVERTER, + FlinkLogicalCorrelate.CONVERTER, + FlinkLogicalIntersect.CONVERTER, + FlinkLogicalJoin.CONVERTER, + FlinkLogicalMinus.CONVERTER, + FlinkLogicalSort.CONVERTER, + FlinkLogicalUnion.CONVERTER, + FlinkLogicalValues.CONVERTER, + FlinkLogicalTableSourceScan.CONVERTER, + FlinkLogicalTableFunctionScan.CONVERTER, + FlinkLogicalNativeTableScan.CONVERTER, + FlinkLogicalMatch.CONVERTER + ) + + + /** + * RuleSet to normalize plans for batch / DataSet execution + */ + val DATASET_NORM_RULES: RuleSet = RuleSets.ofList( + // simplify expressions rules + ReduceExpressionsRule.FILTER_INSTANCE, + ReduceExpressionsRule.PROJECT_INSTANCE, + ReduceExpressionsRule.CALC_INSTANCE, + ReduceExpressionsRule.JOIN_INSTANCE, + ProjectToWindowRule.PROJECT, + + // Transform grouping sets + DecomposeGroupingSetRule.INSTANCE, + // Transform window to LogicalWindowAggregate + DataSetLogicalWindowAggregateRule.INSTANCE, + WindowPropertiesRule.INSTANCE, + WindowPropertiesHavingRule.INSTANCE + ) + + /** + * RuleSet to optimize plans for batch / DataSet execution + */ + val DATASET_OPT_RULES: RuleSet = RuleSets.ofList( + // translate to Flink DataSet nodes + DataSetWindowAggregateRule.INSTANCE, + DataSetAggregateRule.INSTANCE, + DataSetDistinctRule.INSTANCE, + DataSetCalcRule.INSTANCE, + DataSetJoinRule.INSTANCE, + DataSetSingleRowJoinRule.INSTANCE, + DataSetScanRule.INSTANCE, + DataSetUnionRule.INSTANCE, + DataSetIntersectRule.INSTANCE, + DataSetMinusRule.INSTANCE, + DataSetSortRule.INSTANCE, + DataSetValuesRule.INSTANCE, + DataSetCorrelateRule.INSTANCE, + BatchTableSourceScanRule.INSTANCE + ) + + /** + * RuleSet to normalize plans for stream / DataStream execution + */ + val DATASTREAM_NORM_RULES: RuleSet = RuleSets.ofList( + // Transform window to LogicalWindowAggregate + DataStreamLogicalWindowAggregateRule.INSTANCE, + WindowPropertiesRule.INSTANCE, + WindowPropertiesHavingRule.INSTANCE, + + // simplify expressions rules + ReduceExpressionsRule.FILTER_INSTANCE, + ReduceExpressionsRule.PROJECT_INSTANCE, + ReduceExpressionsRule.CALC_INSTANCE, + ProjectToWindowRule.PROJECT + ) + + /** + * RuleSet to optimize plans for stream / DataStream execution + */ + val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( + // translate to DataStream nodes + DataStreamSortRule.INSTANCE, + DataStreamGroupAggregateRule.INSTANCE, + DataStreamOverAggregateRule.INSTANCE, + DataStreamGroupWindowAggregateRule.INSTANCE, + DataStreamCalcRule.INSTANCE, + DataStreamScanRule.INSTANCE, + DataStreamUnionRule.INSTANCE, + DataStreamValuesRule.INSTANCE, + DataStreamCorrelateRule.INSTANCE, + DataStreamWindowJoinRule.INSTANCE, + StreamTableSourceScanRule.INSTANCE, + DataStreamMatchRule.INSTANCE + ) + + /** + * RuleSet to decorate plans for stream / DataStream execution + */ + val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList( + // retraction rules + DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE, + DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE, + DataStreamRetractionRules.ACCMODE_INSTANCE + ) + +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala new file mode 100644 index 000000000..8bb052504 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala @@ -0,0 +1,46 @@ +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.{RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.flink.table.plan.nodes.FlinkConventions +import org.apache.flink.table.plan.nodes.datastream.DataStreamMatch +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalMatch +import org.apache.flink.table.plan.schema.RowSchema + +class DataStreamMatchRule + extends ConverterRule( + classOf[FlinkLogicalMatch], + FlinkConventions.LOGICAL, + FlinkConventions.DATASTREAM, + "DataStreamMatchRule") { + + override def convert(rel: RelNode): RelNode = { + val logicalMatch: FlinkLogicalMatch = rel.asInstanceOf[FlinkLogicalMatch] + val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) + val convertInput: RelNode = + RelOptRule.convert(logicalMatch.getInput, FlinkConventions.DATASTREAM) + + new DataStreamMatch( + rel.getCluster, + traitSet, + convertInput, + logicalMatch.getPattern, + logicalMatch.isStrictStart, + logicalMatch.isStrictEnd, + logicalMatch.getPatternDefinitions, + logicalMatch.getMeasures, + logicalMatch.getAfter, + logicalMatch.getSubsets, + logicalMatch.isAllRows, + logicalMatch.getPartitionKeys, + logicalMatch.getOrderKeys, + logicalMatch.getInterval, + new RowSchema(logicalMatch.getRowType), + new RowSchema(logicalMatch.getInput.getRowType)) + } +} + +object DataStreamMatchRule { + val INSTANCE: RelOptRule = new DataStreamMatchRule +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala new file mode 100644 index 000000000..cfaf5fe81 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala @@ -0,0 +1,14 @@ +package org.apache.flink.table.runtime.cepmatch + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * MapFunction convert CRow to Row. + */ +class ConvertToRow extends MapFunction[CRow, Row] { + override def map(value: CRow): Row = { + value.row + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala new file mode 100644 index 000000000..04a365ebb --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala @@ -0,0 +1,40 @@ +package org.apache.flink.table.runtime.cepmatch + +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.types.Row +import org.slf4j.LoggerFactory + +/** + * IterativeConditionRunner with [[Row]] value. + */ +class IterativeConditionRunner( + name: String, + code: String) + extends IterativeCondition[Row] + with Compiler[IterativeCondition[Row]]{ + + val LOG = LoggerFactory.getLogger(this.getClass) + + // IterativeCondition will be serialized as part of state, + // so make function as transient to avoid ClassNotFoundException when restore state, + // see FLINK-6939 for details + @transient private var function: IterativeCondition[Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling IterativeCondition: $name \n\n Code:\n$code") + // We cannot get user's classloader currently, see FLINK-6938 for details + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating IterativeCondition.") + function = clazz.newInstance() + } + + override def filter(value: Row, ctx: IterativeCondition.Context[Row]): Boolean = { + + if (function == null) { + init() + } + + function.filter(value, ctx) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala new file mode 100644 index 000000000..e0af24e93 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala @@ -0,0 +1,99 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.calcite.rel.RelCollation +import org.apache.calcite.rex.RexNode +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.codegen.MatchCodeGenerator +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * An util class to generate match functions. + * 1。IterativeCondition + * 2。PatternSelectFunction + * 3。PatternFlatSelectFunction + */ +object MatchUtil { + + private[flink] def generateIterativeCondition( + config: TableConfig, + inputType: RowSchema, + patternName: String, + patternNames: Seq[String], + patternDefinition: RexNode, + inputTypeInfo: TypeInformation[_]): IterativeCondition[Row] = { + + val generator = new MatchCodeGenerator( + config, false, inputTypeInfo, patternNames, true, Some(patternName)) + val condition = generator.generateExpression(patternDefinition) + val body = + s""" + |${condition.code} + |return ${condition.resultTerm}; + |""".stripMargin + + val genCondition = generator.generateIterativeCondition("MatchRecognizeCondition", body) + new IterativeConditionRunner(genCondition.name, genCondition.code) + } + + private[flink] def generatePatternSelectFunction( + config: TableConfig, + returnType: RowSchema, + patternNames: Seq[String], + partitionKeys: util.List[RexNode], + measures: util.Map[String, RexNode], + inputTypeInfo: TypeInformation[_]): PatternSelectFunction[Row, CRow] = { + + val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) + + val resultExpression = generator.generateSelectOutputExpression( + partitionKeys, + measures, + returnType) + val body = + s""" + |${resultExpression.code} + |return ${resultExpression.resultTerm}; + |""".stripMargin + + generator.addReusableStatements() + val genFunction = generator.generatePatternSelectFunction( + "MatchRecognizePatternSelectFunction", + body) + new PatternSelectFunctionRunner(genFunction.name, genFunction.code) + } + + private[flink] def generatePatternFlatSelectFunction( + config: TableConfig, + returnType: RowSchema, + patternNames: Seq[String], + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + measures: util.Map[String, RexNode], + inputTypeInfo: TypeInformation[_]): PatternFlatSelectFunction[Row, CRow] = { + + val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) + + val resultExpression = generator.generateFlatSelectOutputExpression( + partitionKeys, + orderKeys, + measures, + returnType) + val body = + s""" + |${resultExpression.code} + |""".stripMargin + + generator.addReusableStatements() + val genFunction = generator.generatePatternFlatSelectFunction( + "MatchRecognizePatternFlatSelectFunction", + body) + new PatternFlatSelectFunctionRunner(genFunction.name, genFunction.code) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala new file mode 100644 index 000000000..f72296324 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala @@ -0,0 +1,47 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.flink.cep.PatternFlatSelectFunction +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.CRowWrappingCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.slf4j.LoggerFactory + +/** + * PatternFlatSelectFunctionRunner with [[Row]] input and [[CRow]] output. + */ +class PatternFlatSelectFunctionRunner( + name: String, + code: String) + extends PatternFlatSelectFunction[Row, CRow] + with Compiler[PatternFlatSelectFunction[Row, Row]] { + + val LOG = LoggerFactory.getLogger(this.getClass) + + private var cRowWrapper: CRowWrappingCollector = _ + + private var function: PatternFlatSelectFunction[Row, Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling PatternFlatSelectFunction: $name \n\n Code:\n$code") + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating PatternFlatSelectFunction.") + function = clazz.newInstance() + + this.cRowWrapper = new CRowWrappingCollector() + } + + override def flatSelect( + pattern: util.Map[String, util.List[Row]], + out: Collector[CRow]): Unit = { + if (function == null) { + init() + } + + cRowWrapper.out = out + function.flatSelect(pattern, cRowWrapper) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala new file mode 100644 index 000000000..d84c9e61d --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala @@ -0,0 +1,45 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.flink.cep.PatternSelectFunction +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.slf4j.LoggerFactory + +/** + * PatternSelectFunctionRunner with [[Row]] input and [[CRow]] output. + */ +class PatternSelectFunctionRunner( + name: String, + code: String) + extends PatternSelectFunction[Row, CRow] + with Compiler[PatternSelectFunction[Row, Row]] { + + val LOG = LoggerFactory.getLogger(this.getClass) + + private var outCRow: CRow = _ + + private var function: PatternSelectFunction[Row, Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling PatternSelectFunction: $name \n\n Code:\n$code") + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating PatternSelectFunction.") + function = clazz.newInstance() + } + + override def select(pattern: util.Map[String, util.List[Row]]): CRow = { + if (outCRow == null) { + outCRow = new CRow(null, true) + } + + if (function == null) { + init() + } + + outCRow.row = function.select(pattern) + outCRow + } +} diff --git a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala new file mode 100644 index 000000000..f89c41d71 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala @@ -0,0 +1,534 @@ +package org.apache.flink.table.validate + +import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes, SqlTypeTransforms} +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable} +import org.apache.calcite.sql._ +import org.apache.flink.table.api._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.functions.sql.ScalarSqlFunctions +import org.apache.flink.table.functions.utils.{AggSqlFunction, ScalarSqlFunction, TableSqlFunction} +import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} + +import _root_.scala.collection.JavaConversions._ +import _root_.scala.collection.mutable +import _root_.scala.util.{Failure, Success, Try} + +/** + * A catalog for looking up (user-defined) functions, used during validation phases + * of both Table API and SQL API. + */ +class FunctionCatalog { + + private val functionBuilders = mutable.HashMap.empty[String, Class[_]] + private val sqlFunctions = mutable.ListBuffer[SqlFunction]() + + def registerFunction(name: String, builder: Class[_]): Unit = + functionBuilders.put(name.toLowerCase, builder) + + def registerSqlFunction(sqlFunction: SqlFunction): Unit = { + sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName) + sqlFunctions += sqlFunction + } + + def getUserDefinedFunctions: Seq[String] = { + sqlFunctions.map(_.getName) + } + + def getSqlOperatorTable: SqlOperatorTable = + ChainedSqlOperatorTable.of( + new BasicOperatorTable(), + new ListSqlOperatorTable(sqlFunctions) + ) + + /** + * Lookup and create an expression if we find a match. + */ + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + val funcClass = functionBuilders + .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name")) + + // Instantiate a function using the provided `children` + funcClass match { + + // user-defined scalar function call + case sf if classOf[ScalarFunction].isAssignableFrom(sf) => + val scalarSqlFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[ScalarSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined scalar function: $name")) + .asInstanceOf[ScalarSqlFunction] + ScalarFunctionCall(scalarSqlFunction.getScalarFunction, children) + + // user-defined table function call + case tf if classOf[TableFunction[_]].isAssignableFrom(tf) => + val tableSqlFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined table function: $name")) + .asInstanceOf[TableSqlFunction] + val typeInfo = tableSqlFunction.getRowTypeInfo + val function = tableSqlFunction.getTableFunction + TableFunctionCall(name, function, children, typeInfo) + + // user-defined aggregate function call + case af if classOf[AggregateFunction[_, _]].isAssignableFrom(af) => + val aggregateFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[AggSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined table function: $name")) + .asInstanceOf[AggSqlFunction] + val function = aggregateFunction.getFunction + val returnType = aggregateFunction.returnType + val accType = aggregateFunction.accType + AggFunctionCall(function, returnType, accType, children) + + // general expression call + case expression if classOf[Expression].isAssignableFrom(expression) => + // try to find a constructor accepts `Seq[Expression]` + Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match { + case Success(seqCtor) => + Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(_) => + Try(funcClass.getDeclaredConstructor(classOf[Expression], classOf[Seq[_]])) match { + case Success(ctor) => + Try(ctor.newInstance(children.head, children.tail).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(_) => + val childrenClass = Seq.fill(children.length)(classOf[Expression]) + // try to find a constructor matching the exact number of children + Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match { + case Success(ctor) => + Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(exception) => throw ValidationException(exception.getMessage) + } + case Failure(_) => + throw ValidationException( + s"Invalid number of arguments for function $funcClass") + } + } + } + case _ => + throw ValidationException("Unsupported function.") + } + } + + /** + * Drop a function and return if the function existed. + */ + def dropFunction(name: String): Boolean = + functionBuilders.remove(name.toLowerCase).isDefined + + /** + * Drop all registered functions. + */ + def clear(): Unit = functionBuilders.clear() +} + +object FunctionCatalog { + + val builtInFunctions: Map[String, Class[_]] = Map( + + // logic + "and" -> classOf[And], + "or" -> classOf[Or], + "not" -> classOf[Not], + "equals" -> classOf[EqualTo], + "greaterThan" -> classOf[GreaterThan], + "greaterThanOrEqual" -> classOf[GreaterThanOrEqual], + "lessThan" -> classOf[LessThan], + "lessThanOrEqual" -> classOf[LessThanOrEqual], + "notEquals" -> classOf[NotEqualTo], + "in" -> classOf[In], + "isNull" -> classOf[IsNull], + "isNotNull" -> classOf[IsNotNull], + "isTrue" -> classOf[IsTrue], + "isFalse" -> classOf[IsFalse], + "isNotTrue" -> classOf[IsNotTrue], + "isNotFalse" -> classOf[IsNotFalse], + "if" -> classOf[If], + + // aggregate functions + "avg" -> classOf[Avg], + "count" -> classOf[Count], + "max" -> classOf[Max], + "min" -> classOf[Min], + "sum" -> classOf[Sum], + "sum0" -> classOf[Sum0], + "stddevPop" -> classOf[StddevPop], + "stddevSamp" -> classOf[StddevSamp], + "varPop" -> classOf[VarPop], + "varSamp" -> classOf[VarSamp], + "collect" -> classOf[Collect], + + // string functions + "charLength" -> classOf[CharLength], + "initCap" -> classOf[InitCap], + "like" -> classOf[Like], + "concat" -> classOf[Plus], + "lower" -> classOf[Lower], + "lowerCase" -> classOf[Lower], + "similar" -> classOf[Similar], + "substring" -> classOf[Substring], + "trim" -> classOf[Trim], + "upper" -> classOf[Upper], + "upperCase" -> classOf[Upper], + "position" -> classOf[Position], + "overlay" -> classOf[Overlay], + "concat" -> classOf[Concat], + "concat_ws" -> classOf[ConcatWs], + "lpad" -> classOf[Lpad], + "rpad" -> classOf[Rpad], + + // math functions + "plus" -> classOf[Plus], + "minus" -> classOf[Minus], + "divide" -> classOf[Div], + "times" -> classOf[Mul], + "abs" -> classOf[Abs], + "ceil" -> classOf[Ceil], + "exp" -> classOf[Exp], + "floor" -> classOf[Floor], + "log10" -> classOf[Log10], + "ln" -> classOf[Ln], + "power" -> classOf[Power], + "mod" -> classOf[Mod], + "sqrt" -> classOf[Sqrt], + "minusPrefix" -> classOf[UnaryMinus], + "sin" -> classOf[Sin], + "cos" -> classOf[Cos], + "tan" -> classOf[Tan], + "cot" -> classOf[Cot], + "asin" -> classOf[Asin], + "acos" -> classOf[Acos], + "atan" -> classOf[Atan], + "degrees" -> classOf[Degrees], + "radians" -> classOf[Radians], + "sign" -> classOf[Sign], + "round" -> classOf[Round], + "pi" -> classOf[Pi], + "e" -> classOf[E], + "rand" -> classOf[Rand], + "randInteger" -> classOf[RandInteger], + "bin" -> classOf[Bin], + + // temporal functions + "extract" -> classOf[Extract], + "currentDate" -> classOf[CurrentDate], + "currentTime" -> classOf[CurrentTime], + "currentTimestamp" -> classOf[CurrentTimestamp], + "localTime" -> classOf[LocalTime], + "localTimestamp" -> classOf[LocalTimestamp], + "quarter" -> classOf[Quarter], + "temporalOverlaps" -> classOf[TemporalOverlaps], + "dateTimePlus" -> classOf[Plus], + "dateFormat" -> classOf[DateFormat], + + // item + "at" -> classOf[ItemAt], + + // cardinality + "cardinality" -> classOf[Cardinality], + + // array + "array" -> classOf[ArrayConstructor], + "element" -> classOf[ArrayElement], + + // map + "map" -> classOf[MapConstructor], + + // row + "row" -> classOf[RowConstructor], + + // window properties + "start" -> classOf[WindowStart], + "end" -> classOf[WindowEnd], + + // ordering + "asc" -> classOf[Asc], + "desc" -> classOf[Desc], + + // crypto hash + "md5" -> classOf[Md5], + "sha1" -> classOf[Sha1], + "sha256" -> classOf[Sha256] + ) + + /** + * Create a new function catalog with built-in functions. + */ + def withBuiltIns: FunctionCatalog = { + val catalog = new FunctionCatalog() + builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) } + catalog + } +} + +class BasicOperatorTable extends ReflectiveSqlOperatorTable { + + /** + * List of supported SQL operators / functions. + * + * This list should be kept in sync with [[SqlStdOperatorTable]]. + */ + private val builtInSqlOperators: Seq[SqlOperator] = Seq( + // SET OPERATORS + SqlStdOperatorTable.UNION, + SqlStdOperatorTable.UNION_ALL, + SqlStdOperatorTable.EXCEPT, + SqlStdOperatorTable.EXCEPT_ALL, + SqlStdOperatorTable.INTERSECT, + SqlStdOperatorTable.INTERSECT_ALL, + // BINARY OPERATORS + SqlStdOperatorTable.AND, + SqlStdOperatorTable.AS, + SqlStdOperatorTable.CONCAT, + SqlStdOperatorTable.DIVIDE, + SqlStdOperatorTable.DIVIDE_INTEGER, + SqlStdOperatorTable.DOT, + SqlStdOperatorTable.EQUALS, + SqlStdOperatorTable.GREATER_THAN, + SqlStdOperatorTable.IS_DISTINCT_FROM, + SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, + SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, + SqlStdOperatorTable.LESS_THAN, + SqlStdOperatorTable.LESS_THAN_OR_EQUAL, + SqlStdOperatorTable.MINUS, + SqlStdOperatorTable.MULTIPLY, + SqlStdOperatorTable.NOT_EQUALS, + SqlStdOperatorTable.OR, + SqlStdOperatorTable.PLUS, + SqlStdOperatorTable.DATETIME_PLUS, + // POSTFIX OPERATORS + SqlStdOperatorTable.DESC, + SqlStdOperatorTable.NULLS_FIRST, + SqlStdOperatorTable.IS_NOT_NULL, + SqlStdOperatorTable.IS_NULL, + SqlStdOperatorTable.IS_NOT_TRUE, + SqlStdOperatorTable.IS_TRUE, + SqlStdOperatorTable.IS_NOT_FALSE, + SqlStdOperatorTable.IS_FALSE, + SqlStdOperatorTable.IS_NOT_UNKNOWN, + SqlStdOperatorTable.IS_UNKNOWN, + // PREFIX OPERATORS + SqlStdOperatorTable.NOT, + SqlStdOperatorTable.UNARY_MINUS, + SqlStdOperatorTable.UNARY_PLUS, + // GROUPING FUNCTIONS + SqlStdOperatorTable.GROUP_ID, + SqlStdOperatorTable.GROUPING, + SqlStdOperatorTable.GROUPING_ID, + // AGGREGATE OPERATORS + SqlStdOperatorTable.SUM, + SqlStdOperatorTable.SUM0, + SqlStdOperatorTable.COUNT, + SqlStdOperatorTable.COLLECT, + SqlStdOperatorTable.MIN, + SqlStdOperatorTable.MAX, + SqlStdOperatorTable.AVG, + SqlStdOperatorTable.STDDEV_POP, + SqlStdOperatorTable.STDDEV_SAMP, + SqlStdOperatorTable.VAR_POP, + SqlStdOperatorTable.VAR_SAMP, + // ARRAY OPERATORS + SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, + SqlStdOperatorTable.ELEMENT, + // MAP OPERATORS + SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR, + // ARRAY MAP SHARED OPERATORS + SqlStdOperatorTable.ITEM, + SqlStdOperatorTable.CARDINALITY, + // SPECIAL OPERATORS + SqlStdOperatorTable.ROW, + SqlStdOperatorTable.OVERLAPS, + SqlStdOperatorTable.LITERAL_CHAIN, + SqlStdOperatorTable.BETWEEN, + SqlStdOperatorTable.SYMMETRIC_BETWEEN, + SqlStdOperatorTable.NOT_BETWEEN, + SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN, + SqlStdOperatorTable.NOT_LIKE, + SqlStdOperatorTable.LIKE, + SqlStdOperatorTable.NOT_SIMILAR_TO, + SqlStdOperatorTable.SIMILAR_TO, + SqlStdOperatorTable.CASE, + SqlStdOperatorTable.REINTERPRET, + SqlStdOperatorTable.EXTRACT, + SqlStdOperatorTable.IN, + // FUNCTIONS + SqlStdOperatorTable.SUBSTRING, + SqlStdOperatorTable.OVERLAY, + SqlStdOperatorTable.TRIM, + SqlStdOperatorTable.POSITION, + SqlStdOperatorTable.CHAR_LENGTH, + SqlStdOperatorTable.CHARACTER_LENGTH, + SqlStdOperatorTable.UPPER, + SqlStdOperatorTable.LOWER, + SqlStdOperatorTable.INITCAP, + SqlStdOperatorTable.POWER, + SqlStdOperatorTable.SQRT, + SqlStdOperatorTable.MOD, + SqlStdOperatorTable.LN, + SqlStdOperatorTable.LOG10, + SqlStdOperatorTable.ABS, + SqlStdOperatorTable.EXP, + SqlStdOperatorTable.NULLIF, + SqlStdOperatorTable.COALESCE, + SqlStdOperatorTable.FLOOR, + SqlStdOperatorTable.CEIL, + SqlStdOperatorTable.LOCALTIME, + SqlStdOperatorTable.LOCALTIMESTAMP, + SqlStdOperatorTable.CURRENT_TIME, + SqlStdOperatorTable.CURRENT_TIMESTAMP, + SqlStdOperatorTable.CURRENT_DATE, + ScalarSqlFunctions.DATE_FORMAT, + SqlStdOperatorTable.CAST, + SqlStdOperatorTable.EXTRACT, + SqlStdOperatorTable.QUARTER, + SqlStdOperatorTable.SCALAR_QUERY, + SqlStdOperatorTable.EXISTS, + SqlStdOperatorTable.SIN, + SqlStdOperatorTable.COS, + SqlStdOperatorTable.TAN, + SqlStdOperatorTable.COT, + SqlStdOperatorTable.ASIN, + SqlStdOperatorTable.ACOS, + SqlStdOperatorTable.ATAN, + SqlStdOperatorTable.DEGREES, + SqlStdOperatorTable.RADIANS, + SqlStdOperatorTable.SIGN, + SqlStdOperatorTable.ROUND, + SqlStdOperatorTable.PI, + ScalarSqlFunctions.E, + SqlStdOperatorTable.RAND, + SqlStdOperatorTable.RAND_INTEGER, + ScalarSqlFunctions.CONCAT, + ScalarSqlFunctions.CONCAT_WS, + ScalarSqlFunctions.BIN, + SqlStdOperatorTable.TIMESTAMP_ADD, + ScalarSqlFunctions.LOG, + ScalarSqlFunctions.LPAD, + ScalarSqlFunctions.RPAD, + ScalarSqlFunctions.MD5, + ScalarSqlFunctions.SHA1, + ScalarSqlFunctions.SHA256, + + // EXTENSIONS + BasicOperatorTable.TUMBLE, + BasicOperatorTable.HOP, + BasicOperatorTable.SESSION, + BasicOperatorTable.TUMBLE_START, + BasicOperatorTable.TUMBLE_END, + BasicOperatorTable.HOP_START, + BasicOperatorTable.HOP_END, + BasicOperatorTable.SESSION_START, + BasicOperatorTable.SESSION_END, + BasicOperatorTable.TUMBLE_PROCTIME, + BasicOperatorTable.TUMBLE_ROWTIME, + BasicOperatorTable.HOP_PROCTIME, + BasicOperatorTable.HOP_ROWTIME, + BasicOperatorTable.SESSION_PROCTIME, + BasicOperatorTable.SESSION_ROWTIME, + + // MATCH_RECOGNIZE + SqlStdOperatorTable.FIRST, + SqlStdOperatorTable.LAST, + SqlStdOperatorTable.PREV, + SqlStdOperatorTable.NEXT, + SqlStdOperatorTable.CLASSIFIER, + SqlStdOperatorTable.MATCH_NUMBER, + SqlStdOperatorTable.FINAL, + SqlStdOperatorTable.RUNNING + ) + + builtInSqlOperators.foreach(register) +} + +object BasicOperatorTable { + + /** + * We need custom group auxiliary functions in order to support nested windows. + */ + + val TUMBLE: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.TUMBLE, + null, + OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + TUMBLE_START, + TUMBLE_END, + TUMBLE_ROWTIME, + TUMBLE_PROCTIME) + } + val TUMBLE_START: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_START) + val TUMBLE_END: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_END) + val TUMBLE_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "TUMBLE_ROWTIME", + SqlKind.OTHER_FUNCTION, + TUMBLE, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + TUMBLE.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val TUMBLE_PROCTIME: SqlGroupedWindowFunction = + TUMBLE.auxiliary("TUMBLE_PROCTIME", SqlKind.OTHER_FUNCTION) + + val HOP: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.HOP, + null, + OperandTypes.or( + OperandTypes.DATETIME_INTERVAL_INTERVAL, + OperandTypes.DATETIME_INTERVAL_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + HOP_START, + HOP_END, + HOP_ROWTIME, + HOP_PROCTIME) + } + val HOP_START: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_START) + val HOP_END: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_END) + val HOP_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "HOP_ROWTIME", + SqlKind.OTHER_FUNCTION, + HOP, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + HOP.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val HOP_PROCTIME: SqlGroupedWindowFunction = HOP.auxiliary("HOP_PROCTIME", SqlKind.OTHER_FUNCTION) + + val SESSION: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.SESSION, + null, + OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + SESSION_START, + SESSION_END, + SESSION_ROWTIME, + SESSION_PROCTIME) + } + val SESSION_START: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_START) + val SESSION_END: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_END) + val SESSION_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "SESSION_ROWTIME", + SqlKind.OTHER_FUNCTION, + SESSION, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + SESSION.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val SESSION_PROCTIME: SqlGroupedWindowFunction = + SESSION.auxiliary("SESSION_PROCTIME", SqlKind.OTHER_FUNCTION) + +} diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index e29277fee..9771f6dd7 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -39,6 +39,51 @@ public class SideSqlExecTest { + /** + * 参考阿里Blink的cep sql语法,文档https://help.aliyun.com/document_detail/73845.html?spm=a2c4g.11186623.6.637.5cba27efFHjOSs + * @throws Exception + */ + @Test + public void testCepSql() throws Exception { + List paramList = Lists.newArrayList(); + paramList.add("-sql"); + String sqlContext = "CREATE table source(" + + "name varchar, " + + "price float, " + + "tax float, " + + "tstamp timestamp) " + + "with (" + + " type = 'kafka09',bootstrapServers = 'kudu1:9092',zookeeperQuorum = '172.16.8.107:2181/kafka', offsetReset = 'latest',topic = 'tranflow_input',parallelism = '1' " + + ");" + + "CREATE table sink(" + + "start_tstamp timestamp, " + + "bottom_tstamp timestamp, " + + "end_tstamp timestamp, " + + "bottom_total float, " + + "end_total float ) " + + "with (" + + " type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'max_deposit_acct_base',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1' " + + ");" + + "insert into sink " + + "select * from source " + + "MATCH_RECOGNIZE (\n" + + " MEASURES\n" + + " STRT.tstamp AS start_tstamp,\n" + + " LAST(DOWN.tstamp) AS bottom_tstamp,\n" + + " LAST(UP.tstamp) AS end_tstamp,\n" + + " FIRST(DOWN.price + DOWN.tax + 1) AS bottom_total,\n" + + " FIRST(UP.price + UP.tax) AS end_total" + + " ONE ROW PER MATCH\n" + + " PATTERN (STRT DOWN+ UP+)\n" + + " DEFINE\n" + + " DOWN AS DOWN.price < PREV(DOWN.price),\n" + + " UP AS UP.price > PREV(UP.price) AND UP.tax > LAST(DOWN.tax)\n" + + ") AS T" + ; + test(sqlContext); + } + + @Test public void testRunSideSql() throws Exception { //String runParam = "-sql CREATE+TABLE+MyTable(channel+STRING%2c+pv+INT%2c+xctime+bigint%2c+timeLeng+as+CHARACTER_LENGTH(channel)%2c++WATERMARK+FOR+xctime+AS+withOffset(xctime%2c+1000))+WITH+(+type%3d%27kafka09%27%2c+bootstrapServers%3d%27172.16.8.198%3a9092%27%2c+offsetReset%3d%27latest%27%2ctopic%3d%27nbTest1%27)%3bCREATE+TABLE+MyResult(channel+STRING%2c+pv+INT)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3bcreate+table+sideTable(channel+String%2c+count+int%2c+PERIOD+FOR+SYSTEM_TIME)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3binsert+into+MyResult+select+a.channel%2cb.pv+from+MyTable+a+join+sideTable+b+on+a.channel%3db.channel%3b -name xc -localSqlPluginPath D:\\gitspace\\flink-sql-plugin\\plugins -mode local -remoteSqlPluginPath /opt/dtstack/flinkplugin -confProp %7b%22time.characteristic%22%3a%22EventTime%22%7d -addjar %5b%22D%3a%5c%5cgitspace%5c%5crdos-execution-engine%5c%5c..%5c%5ctmp140%5c%5cflink14Test-1.0-SNAPSHOT.jar%22%5d"; @@ -228,7 +273,8 @@ public void test(String sql) throws Exception { paramList.add("-name"); paramList.add("xc"); paramList.add("-localSqlPluginPath"); - paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); +// paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); + paramList.add("/Users/meitu/code/github/flinkStreamSQL/plugins"); paramList.add("-mode"); paramList.add("local"); paramList.add("-addjar"); 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 26ab2da73..c418360cb 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.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -113,6 +114,8 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.BINARY; }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; + }else if(fieldType.equals(Timestamp.class.getName())){ + tmpFieldsType[i] = Types.TIMESTAMP; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 379b9a53ea9d3c3b8286d8d30d0c65f0444d2a7a Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 19:07:10 +0800 Subject: [PATCH 05/78] extract outputformat --- .../java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java | 6 +++++- .../com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 2 +- .../main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java | 5 +++-- .../sql/sink/rdb/{format => }/RetractJDBCOutputFormat.java | 3 +-- .../flink/sql/sink/sqlserver/SqlserverOutputFormat.java | 1 - 5 files changed, 10 insertions(+), 7 deletions(-) rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{format => }/RetractJDBCOutputFormat.java (99%) 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 6d4d86d23..b872ecccd 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,7 +23,6 @@ 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; @@ -79,5 +78,10 @@ public String getDriverName() { return MYSQL_DRIVER; } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + } 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 54ea47a18..04ad57306 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 @@ -213,7 +213,7 @@ private void loadData(Map>> tmpCache) throws SQ } } - public int getFetchSize(){ + public int getFetchSize() { return 1000; } 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 d587639aa..a3611bfc6 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 @@ -99,8 +99,6 @@ public RichSinkFunction createJdbcSinkFunc() { return outputFormatSinkFunc; } - public abstract RetractJDBCOutputFormat getOutputFormat(); - @Override public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { @@ -252,4 +250,7 @@ public void setDbType(String dbType) { public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); public abstract String getDriverName(); + + public abstract RetractJDBCOutputFormat getOutputFormat(); + } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java similarity index 99% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 8f5329796..a003cb69a 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package com.dtstack.flink.sql.sink.rdb.format; +package com.dtstack.flink.sql.sink.rdb; -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; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java index 21d1f3bd1..7708ba163 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java @@ -17,7 +17,6 @@ */ 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; From 7a498c9ef808952b0b2213fa2bfacf0b7b3448bd Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 3 Dec 2018 09:47:19 +0800 Subject: [PATCH 06/78] add output package --- .../java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java | 5 ----- .../sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java | 4 ++-- 2 files changed, 2 insertions(+), 7 deletions(-) rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java (99%) 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 b872ecccd..d045729e8 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 @@ -78,10 +78,5 @@ public String getDriverName() { return MYSQL_DRIVER; } - @Override - public RetractJDBCOutputFormat getOutputFormat() { - return new RetractJDBCOutputFormat(); - } - } 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 99% 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 a003cb69a..382b687e1 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,8 +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.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; @@ -27,7 +28,6 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.sql.*; import java.util.*; import java.io.IOException; From 72df82e3ea1d141d2d7ba5599b43f8a1f169e347 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 3 Dec 2018 16:33:08 +0800 Subject: [PATCH 07/78] case sensitive --- .../flink/sql/sink/redis/table/RedisSinkParser.java | 10 +++++----- .../flink/sql/sink/redis/table/RedisTableInfo.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) 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 6960dc249..5754bdfee 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 @@ -37,11 +37,11 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 4 Dec 2018 10:18:08 +0800 Subject: [PATCH 08/78] oracle side and sink --- oracle/oracle-side/oracle-all-side/pom.xml | 91 +++++++ .../sql/side/oracle/OracleAllReqRow.java | 62 +++++ .../sql/side/oracle/OracleAllSideInfo.java | 33 +++ oracle/oracle-side/oracle-async-side/pom.xml | 92 +++++++ .../sql/side/oracle/OracleAsyncReqRow.java | 66 +++++ .../sql/side/oracle/OracleAsyncSideInfo.java | 35 +++ oracle/oracle-side/oracle-side-core/pom.xml | 18 ++ .../side/oracle/table/OracleSideParser.java | 35 +++ oracle/oracle-side/pom.xml | 33 +++ oracle/oracle-sink/pom.xml | 90 +++++++ .../sql/sink/oracle/OracleOutputFormat.java | 105 ++++++++ .../flink/sql/sink/oracle/OracleSink.java | 171 +++++++++++++ .../sink/oracle/table/OracleSinkParser.java | 41 +++ oracle/pom.xml | 39 +++ pom.xml | 1 + .../flink/sql/side/rdb/all/RdbAllReqRow.java | 11 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 9 +- .../flink/sql/side/rdb/util/MathUtil.java | 236 ++++++++++++++++++ .../flink/sql/side/rdb/util/SwitchUtil.java | 58 +++++ 19 files changed, 1223 insertions(+), 3 deletions(-) create mode 100644 oracle/oracle-side/oracle-all-side/pom.xml create mode 100644 oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java create mode 100644 oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java create mode 100644 oracle/oracle-side/oracle-async-side/pom.xml create mode 100644 oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java create mode 100644 oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java create mode 100644 oracle/oracle-side/oracle-side-core/pom.xml create mode 100644 oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java create mode 100644 oracle/oracle-side/pom.xml create mode 100644 oracle/oracle-sink/pom.xml create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java create mode 100644 oracle/pom.xml create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java diff --git a/oracle/oracle-side/oracle-all-side/pom.xml b/oracle/oracle-side/oracle-all-side/pom.xml new file mode 100644 index 000000000..458077376 --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/pom.xml @@ -0,0 +1,91 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.oracle + 1.0-SNAPSHOT + oracle-all-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.oracle.core + ${sql.side.oracle.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/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java new file mode 100644 index 000000000..b6779529e --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java @@ -0,0 +1,62 @@ +/** + * 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.oracle; + + +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 OracleAllReqRow extends RdbAllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(OracleAllReqRow.class); + + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + public OracleAllReqRow(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(ORACLE_DRIVER); + Map addParams = Maps.newHashMap(); + 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/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java new file mode 100644 index 000000000..4063bf9bb --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.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.side.oracle; + +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 OracleAllSideInfo extends RdbAllSideInfo { + + public OracleAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/oracle/oracle-side/oracle-async-side/pom.xml b/oracle/oracle-side/oracle-async-side/pom.xml new file mode 100644 index 000000000..3a4b36559 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/pom.xml @@ -0,0 +1,92 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.oracle + 1.0-SNAPSHOT + oracle-async-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.oracle.core + ${sql.side.oracle.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/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java new file mode 100644 index 000000000..c68850d21 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.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.side.oracle; + +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.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.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 OracleAsyncReqRow extends RdbAsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(OracleAsyncReqRow.class); + + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + public OracleAsyncReqRow(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", ORACLE_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/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java new file mode 100644 index 000000000..7ff63f3d9 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.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.oracle; + +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 OracleAsyncSideInfo extends RdbAsyncSideInfo { + + public OracleAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/oracle/oracle-side/oracle-side-core/pom.xml b/oracle/oracle-side/oracle-side-core/pom.xml new file mode 100644 index 000000000..cc954d9fd --- /dev/null +++ b/oracle/oracle-side/oracle-side-core/pom.xml @@ -0,0 +1,18 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.oracle.core + 1.0-SNAPSHOT + jar + oracle-side-core + + \ No newline at end of file diff --git a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java new file mode 100644 index 000000000..f9124bd10 --- /dev/null +++ b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.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.oracle.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + + +public class OracleSideParser extends RdbSideParser { + private static final String CURR_TYPE = "oracle"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + oracleTableInfo.setType(CURR_TYPE); + return oracleTableInfo; + } +} diff --git a/oracle/oracle-side/pom.xml b/oracle/oracle-side/pom.xml new file mode 100644 index 000000000..e0ffe547b --- /dev/null +++ b/oracle/oracle-side/pom.xml @@ -0,0 +1,33 @@ + + + + sql.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.oracle + 1.0-SNAPSHOT + + oracle-side-core + oracle-all-side + oracle-async-side + + pom + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + \ No newline at end of file diff --git a/oracle/oracle-sink/pom.xml b/oracle/oracle-sink/pom.xml new file mode 100644 index 000000000..c5807f806 --- /dev/null +++ b/oracle/oracle-sink/pom.xml @@ -0,0 +1,90 @@ + + + + sql.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.oracle + 1.0-SNAPSHOT + oracle-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/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java new file mode 100644 index 000000000..dbbeea270 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java @@ -0,0 +1,105 @@ +/* + * 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.oracle; + +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; +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 { + + + @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/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java new file mode 100644 index 000000000..de1ffe8e5 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -0,0 +1,171 @@ +/** + * 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.oracle; + +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.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleSink extends RdbSink implements IStreamSinkGener { + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + @Override + public String getDriverName() { + return ORACLE_DRIVER; + } + + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new OracleOutputFormat(); + } + + @Override + public void buildSql(String tableName, List fields) { + buildInsertSql(tableName, 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 "\""; + } + + +} diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java new file mode 100644 index 000000000..6db2c9c06 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.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.oracle.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 OracleSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "oracle"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlserverTableInfo.setType(CURR_TYPE); + return sqlserverTableInfo; + } +} diff --git a/oracle/pom.xml b/oracle/pom.xml new file mode 100644 index 000000000..60565a149 --- /dev/null +++ b/oracle/pom.xml @@ -0,0 +1,39 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.oracle + 1.0-SNAPSHOT + + oracle-side + oracle-sink + + pom + + + 12.2.0.1 + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + + com.github.noraui + ojdbc8 + ${ojdbc.version} + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index bde73d6df..a31cc7392 100644 --- a/pom.xml +++ b/pom.xml @@ -18,6 +18,7 @@ launcher rdb sqlserver + oracle pom 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 04ad57306..fce61e238 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 @@ -23,8 +23,11 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.side.rdb.util.MathUtil; +import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; 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; @@ -34,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigDecimal; import java.sql.*; import java.util.Calendar; import java.util.List; @@ -74,6 +78,7 @@ protected Row fillData(Row input, Object sideInput) { if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { obj = ((Timestamp) obj).getTime(); } + row.setField(entry.getKey(), obj); } @@ -194,10 +199,14 @@ private void loadData(Map>> tmpCache) throws SQ statement.setFetchSize(getFetchSize()); ResultSet resultSet = statement.executeQuery(sql); String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + String[] fields = sideInfo.getSideTableInfo().getFieldTypes(); while (resultSet.next()) { Map oneRow = Maps.newHashMap(); for (String fieldName : sideFieldNames) { - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + Object object = resultSet.getObject(fieldName.trim()); + int fieldIndex = sideInfo.getRowTypeInfo().getFieldIndex(fieldName.trim()); + object = SwitchUtil.getTarget(object, fields[fieldIndex]); + oneRow.put(fieldName.trim(), object); } String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); 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 a7e5339e9..2c3d9b3f4 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 @@ -22,9 +22,11 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 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; @@ -34,6 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigInteger; import java.sql.Timestamp; import java.util.Collections; import java.util.List; @@ -120,6 +123,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except 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); @@ -151,10 +155,10 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except public Row fillData(Row input, Object line) { JsonArray jsonArray = (JsonArray) line; Row row = new Row(sideInfo.getOutFieldInfoList().size()); + String[] fields = sideInfo.getSideTableInfo().getFieldTypes(); 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(); } @@ -166,7 +170,8 @@ public Row fillData(Row input, Object line) { if (jsonArray == null) { row.setField(entry.getKey(), null); } else { - row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getKey()]); + row.setField(entry.getKey(), object); } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java new file mode 100644 index 000000000..aa7442586 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -0,0 +1,236 @@ +/* + * 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.util; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; + +/** + * Date: 2017/4/21 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class MathUtil { + + public static Long getLongVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Long.valueOf((String) obj); + } else if (obj instanceof Long) { + return (Long) obj; + } else if (obj instanceof Integer) { + return Long.valueOf(obj.toString()); + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).longValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Long."); + } + + public static Long getLongVal(Object obj, long defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getLongVal(obj); + } + + public static Integer getIntegerVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Integer.valueOf((String) obj); + } else if (obj instanceof Integer) { + return (Integer) obj; + } else if (obj instanceof Long) { + return ((Long) obj).intValue(); + } else if (obj instanceof Double) { + return ((Double) obj).intValue(); + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).intValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Integer."); + } + + public static Integer getIntegerVal(Object obj, int defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getIntegerVal(obj); + } + + public static Float getFloatVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Float.valueOf((String) obj); + } else if (obj instanceof Float) { + return (Float) obj; + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).floatValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Float."); + } + + public static Float getFloatVal(Object obj, float defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getFloatVal(obj); + } + + public static Double getDoubleVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Double.valueOf((String) obj); + } else if (obj instanceof Float) { + return (Double) obj; + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).doubleValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Double."); + } + + public static Double getDoubleVal(Object obj, double defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getDoubleVal(obj); + } + + + public static Boolean getBoolean(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Boolean.valueOf((String) obj); + } else if (obj instanceof Boolean) { + return (Boolean) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Boolean."); + } + + public static Boolean getBoolean(Object obj, boolean defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getBoolean(obj); + } + + public static String getString(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return (String) obj; + } + + return obj.toString(); + } + + public static Byte getByte(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Byte.valueOf((String) obj); + } else if (obj instanceof Byte) { + return (Byte) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Byte."); + } + + public static Short getShort(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Short.valueOf((String) obj); + } else if (obj instanceof Short) { + return (Short) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Short."); + } + + public static BigDecimal getBigDecimal(Object obj) { + if (obj == null) { + return null; + } + if (obj instanceof String) { + return new BigDecimal((String) obj); + } else if (obj instanceof BigDecimal) { + return (BigDecimal) obj; + } else if (obj instanceof BigInteger) { + return new BigDecimal((BigInteger) obj); + } else if (obj instanceof Number) { + return new BigDecimal(((Number) obj).doubleValue()); + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to BigDecimal."); + } + + public static Date getDate(Object obj) { + if (obj == null) { + return null; + } + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + if (obj instanceof String) { + try { + return sdf.parse((String) obj); + } catch (ParseException e) { + e.printStackTrace(); + } + } else if (obj instanceof Date) { + return (Date) obj; + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); + } + + +} \ No newline at end of file diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java new file mode 100644 index 000000000..eadbdc476 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.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.side.rdb.util; + +import java.text.ParseException; + +/** + * Reason: + * Date: 2018/12/3 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SwitchUtil { + public static Object getTarget(Object obj, String targetType) { + targetType = targetType.toLowerCase(); + switch (targetType) { + case "int": + case "integer": + return MathUtil.getIntegerVal(obj); + case "bigint": + return MathUtil.getLongVal(obj); + case "boolean": + return MathUtil.getBoolean(obj); + case "tinyint": + return MathUtil.getByte(obj); + case "smallint": + return MathUtil.getShort(obj); + case "varchar": + return MathUtil.getString(obj); + case "real": + case "float": + return MathUtil.getFloatVal(obj); + case "double": + return MathUtil.getDoubleVal(obj); + case "decimal": + return MathUtil.getBigDecimal(obj); + case "date": + return MathUtil.getDate(obj); + } + return obj; + } +} From 1e9b14dd43a1ac0f0092db8f7d87a75af18d2bf1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 10:42:40 +0800 Subject: [PATCH 09/78] fix abstract tableInfo not Serializable --- .../dtstack/flink/sql/table/TableInfo.java | 26 ++++++++++++++++++- .../flink/sql/launcher/LauncherMain.java | 4 ++- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 ++ .../sql/side/rdb/table/RdbSideTableInfo.java | 1 + 4 files changed, 31 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index 71bea466f..b56b790ea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -22,6 +22,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import java.io.Serializable; import java.util.List; /** @@ -31,7 +32,7 @@ * @author xuchao */ -public abstract class TableInfo { +public abstract class TableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; @@ -121,6 +122,29 @@ public void addFieldType(String fieldType){ fieldTypeList.add(fieldType); } + public void setFields(String[] fields) { + this.fields = fields; + } + + public void setFieldTypes(String[] fieldTypes) { + this.fieldTypes = fieldTypes; + } + + public void setFieldClasses(Class[] fieldClasses) { + this.fieldClasses = fieldClasses; + } + + public List getFieldList() { + return fieldList; + } + + public List getFieldTypeList() { + return fieldTypeList; + } + + public List getFieldClassList() { + return fieldClassList; + } public void finish(){ this.fields = fieldList.toArray(new String[fieldList.size()]); 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 10bc09475..eea52e0af 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 @@ -59,13 +59,15 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ } public static void main(String[] args) throws Exception { - if (args.length==1 && args[0].endsWith(".json")){ + if (args.length == 1 && args[0].endsWith(".json")){ args = parseJson(args); } + LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); 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 index b7ff94ea9..f652bf342 100644 --- 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 @@ -42,6 +42,8 @@ public class RdbAsyncSideInfo extends SideInfo { + private static final long serialVersionUID = 1942629132469918611L; + public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } 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 c08f9a3a9..44d0af11e 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 @@ -28,6 +28,7 @@ * @author maqi */ public class RdbSideTableInfo extends SideTableInfo { + private static final long serialVersionUID = -1L; public static final String URL_KEY = "url"; From 508050c72b9e8cb4b8f5f5d159bb01b4587f5f3c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 11:29:39 +0800 Subject: [PATCH 10/78] modify reqRow to a interface --- .../com/dtstack/flink/sql/side/AllReqRow.java | 4 +- .../dtstack/flink/sql/side/AsyncReqRow.java | 5 +- .../dtstack/flink/sql/side/ISideReqRow.java | 33 +++++++++ .../flink/sql/side/hbase/HbaseAllReqRow.java | 2 +- .../sql/side/hbase/HbaseAsyncReqRow.java | 2 +- .../flink/sql/side/mongo/MongoAllReqRow.java | 2 +- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 2 +- .../flink/sql/side/redis/RedisAllReqRow.java | 29 ++------ .../sql/side/redis/RedisAsyncReqRow.java | 29 ++------ .../sql/side/redis/table/RedisSideReqRow.java | 72 +++++++++++++++++++ 10 files changed, 122 insertions(+), 58 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java create mode 100644 redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java index 303260742..a185da1bd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java @@ -37,7 +37,7 @@ * @author xuchao */ -public abstract class AllReqRow extends RichFlatMapFunction{ +public abstract class AllReqRow extends RichFlatMapFunction implements ISideReqRow { protected SideInfo sideInfo; @@ -48,8 +48,6 @@ public AllReqRow(SideInfo sideInfo){ } - protected abstract Row fillData(Row input, Object sideInput); - protected abstract void initCache() throws SQLException; protected abstract void reloadCache(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index 6617bb29f..6df1af1d5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -40,7 +40,7 @@ * @author xuchao */ -public abstract class AsyncReqRow extends RichAsyncFunction { +public abstract class AsyncReqRow extends RichAsyncFunction implements ISideReqRow { private static final long serialVersionUID = 2098635244857937717L; @@ -79,9 +79,6 @@ protected boolean openCache(){ return sideInfo.getSideCache() != null; } - - protected abstract Row fillData(Row input, Object sideInput); - protected void dealMissKey(Row input, ResultFuture resultFuture){ if(sideInfo.getJoinType() == JoinType.LEFT){ //Reserved left table data diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java new file mode 100644 index 000000000..88066e37f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.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.side; + +import org.apache.flink.types.Row; + +/** + * + * Date: 2018/12/4 + * Company: www.dtstack.com + * @author xuchao + */ +public interface ISideReqRow { + + Row fillData(Row input, Object sideInput); + +} diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index 29fca4bee..de89f337e 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -66,7 +66,7 @@ public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List sideInputList = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ 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 fe227bb6a..6e82e4109 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 @@ -160,7 +160,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } @Override - protected Row fillData(Row input, Object sideInput){ + public Row fillData(Row input, Object sideInput){ List sideInputList = (List) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index e1a398d4c..c1c812cd0 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -80,7 +80,7 @@ public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { 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 fce61e238..c7e37c590 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 @@ -67,7 +67,7 @@ public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List } @Override - protected Row fillData(Row input, Object sideInput) { + public Row fillData(Row input, Object sideInput) { Map cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { 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 e0e0a7d17..54ecf8952 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 @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.redis; import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import org.apache.calcite.sql.JoinType; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; @@ -55,34 +56,16 @@ public class RedisAllReqRow extends AllReqRow{ private AtomicReference>> cacheRef = new AtomicReference<>(); + private RedisSideReqRow redisSideReqRow; + public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RedisAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + this.redisSideReqRow = new RedisSideReqRow(super.sideInfo); } @Override - protected Row fillData(Row input, Object sideInput) { - Map sideInputMap = (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()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(sideInputMap == null){ - row.setField(entry.getKey(), null); - }else{ - String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); - row.setField(entry.getKey(), sideInputMap.get(key)); - } - } - - return row; + public Row fillData(Row input, Object sideInput) { + return redisSideReqRow.fillData(input, sideInput); } @Override 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 82532c527..3f9f40f24 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 @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import io.lettuce.core.KeyValue; import io.lettuce.core.RedisClient; @@ -60,9 +61,11 @@ public class RedisAsyncReqRow extends AsyncReqRow { private RedisSideTableInfo redisSideTableInfo; + private RedisSideReqRow redisSideReqRow; public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RedisAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + redisSideReqRow = new RedisSideReqRow(super.sideInfo); } @Override @@ -108,30 +111,8 @@ private void buildRedisClient(RedisSideTableInfo tableInfo){ } @Override - protected Row fillData(Row input, Object sideInput) { - Map keyValue = (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()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(keyValue == null){ - row.setField(entry.getKey(), null); - }else{ - String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); - row.setField(entry.getKey(), keyValue.get(key)); - } - } - - return row; + public Row fillData(Row input, Object sideInput) { + return redisSideReqRow.fillData(input, sideInput); } @Override diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java new file mode 100644 index 000000000..40dae6c68 --- /dev/null +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java @@ -0,0 +1,72 @@ +/* + * 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.redis.table; + +import com.dtstack.flink.sql.side.ISideReqRow; +import com.dtstack.flink.sql.side.SideInfo; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; + +import java.io.Serializable; +import java.sql.Timestamp; +import java.util.Map; + +/** + * redis fill row data + * Date: 2018/12/4 + * Company: www.dtstack.com + * @author xuchao + */ + +public class RedisSideReqRow implements ISideReqRow, Serializable { + + private static final long serialVersionUID = 3751171828444748982L; + + private SideInfo sideInfo; + + public RedisSideReqRow(SideInfo sideInfo){ + this.sideInfo = sideInfo; + } + + @Override + public Row fillData(Row input, Object sideInput) { + Map sideInputMap = (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()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(sideInputMap == null){ + row.setField(entry.getKey(), null); + }else{ + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), sideInputMap.get(key)); + } + } + + return row; + } +} From be12caaf278758308c2d0a26bbdf1bd3410b0b95 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 4 Dec 2018 17:26:36 +0800 Subject: [PATCH 11/78] the value is case sensitive --- .../com/dtstack/flink/sql/parser/CreateTmpTableParser.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 9bd1374a0..8da89b2be 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -77,7 +78,8 @@ public void parseSql(String sql, SqlTree sqlTree) { parseNode(sqlNode, sqlParseResult); sqlParseResult.setTableName(tableName); - sqlParseResult.setExecSql(selectSql.toUpperCase()); + String transformSelectSql = DtStringUtil.replaceIgnoreQuota(sqlNode.toString(), "`", ""); + sqlParseResult.setExecSql(transformSelectSql); sqlTree.addTmpSql(sqlParseResult); sqlTree.addTmplTableInfo(tableName, sqlParseResult); } else { From d79838612905ed4eda5a3abe23ee1f96b659c344 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:29:58 +0800 Subject: [PATCH 12/78] oracle side and sink and modify jdbc getdate type --- .../sql/sink/oracle/OracleOutputFormat.java | 105 ------------------ .../flink/sql/sink/oracle/OracleSink.java | 8 +- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 3 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- .../flink/sql/side/rdb/util/MathUtil.java | 29 +++-- .../flink/sql/side/rdb/util/SwitchUtil.java | 4 +- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 7 +- .../sql/sink/rdb/table/RdbTableInfo.java | 2 - .../sink/sqlserver/SqlserverOutputFormat.java | 103 ----------------- .../sql/sink/sqlserver/SqlserverSink.java | 3 +- 10 files changed, 40 insertions(+), 226 deletions(-) delete mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java delete mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java deleted file mode 100644 index dbbeea270..000000000 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java +++ /dev/null @@ -1,105 +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.oracle; - -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; -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 { - - - @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/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index de1ffe8e5..4252f025c 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.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.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; @@ -43,7 +44,7 @@ public String getDriverName() { @Override public RetractJDBCOutputFormat getOutputFormat() { - return new OracleOutputFormat(); + return new ExtendOutputFormat(); } @Override @@ -71,7 +72,7 @@ public String buildUpdateSql(String tableName, List fieldNames, Map 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 || keyCols.contains(col)) { continue; } if (fullColumn == null || column.contains(col)) { @@ -152,6 +153,7 @@ public String makeValues(List column) { } sb.append("? " + quoteColumn(column.get(i))); } + sb.append(" FROM DUAL"); return sb.toString(); } 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 c7e37c590..abae0cbb2 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 @@ -62,6 +62,7 @@ public abstract class RdbAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); + public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RdbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -204,7 +205,7 @@ private void loadData(Map>> tmpCache) throws SQ Map oneRow = Maps.newHashMap(); for (String fieldName : sideFieldNames) { Object object = resultSet.getObject(fieldName.trim()); - int fieldIndex = sideInfo.getRowTypeInfo().getFieldIndex(fieldName.trim()); + int fieldIndex = sideInfo.getSideTableInfo().getFieldList().indexOf(fieldName.trim()); object = SwitchUtil.getTarget(object, fields[fieldIndex]); oneRow.put(fieldName.trim(), object); } 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 2c3d9b3f4..73fab0e9f 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 @@ -170,7 +170,7 @@ public Row fillData(Row input, Object line) { if (jsonArray == null) { row.setField(entry.getKey(), null); } else { - Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getKey()]); + Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getValue()]); row.setField(entry.getKey(), object); } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index aa7442586..fd6af5e0b 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -21,9 +21,10 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.sql.Date; +import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.Date; /** * Date: 2017/4/21 @@ -75,6 +76,8 @@ public static Integer getIntegerVal(Object obj) { return ((Double) obj).intValue(); } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).intValue(); + } else if (obj instanceof BigInteger) { + return ((BigInteger) obj).intValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Integer."); @@ -219,18 +222,28 @@ public static Date getDate(Object obj) { if (obj == null) { return null; } - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); if (obj instanceof String) { - try { - return sdf.parse((String) obj); - } catch (ParseException e) { - e.printStackTrace(); - } - } else if (obj instanceof Date) { + return Date.valueOf((String) obj); + } else if (obj instanceof Timestamp) { + return new Date(((Timestamp) obj).getTime()); + }else if (obj instanceof Date){ return (Date) obj; } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } + public static Timestamp getTimestamp(Object obj) { + if (obj == null) { + return null; + } + if (obj instanceof Timestamp) { + return (Timestamp) obj; + } else if (obj instanceof Date) { + return Timestamp.valueOf((String) obj); + } else if (obj instanceof String) { + return Timestamp.valueOf((String) obj); + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); + } } \ No newline at end of file diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index eadbdc476..1d06a4193 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -27,7 +27,7 @@ * @author maqi */ public class SwitchUtil { - public static Object getTarget(Object obj, String targetType) { + public static Object getTarget(Object obj, String targetType) { targetType = targetType.toLowerCase(); switch (targetType) { case "int": @@ -52,6 +52,8 @@ public static Object getTarget(Object obj, String targetType) { return MathUtil.getBigDecimal(obj); case "date": return MathUtil.getDate(obj); + case "timestamp": + return MathUtil.getTimestamp(obj); } return obj; } 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 a3611bfc6..5ee2c44c4 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 @@ -35,6 +35,7 @@ import java.io.Serializable; import java.math.BigDecimal; +import java.sql.Date; import java.sql.Timestamp; import java.sql.Types; import java.util.Arrays; @@ -149,7 +150,9 @@ protected void buildSqlTypes(List fieldTypeArray) { String fieldType = fieldTypeArray.get(i).getName(); if (fieldType.equals(Integer.class.getName())) { tmpFieldsType[i] = Types.INTEGER; - } else if (fieldType.equals(Long.class.getName())) { + }else if (fieldType.equals(Boolean.class.getName())) { + tmpFieldsType[i] = Types.BOOLEAN; + }else if (fieldType.equals(Long.class.getName())) { tmpFieldsType[i] = Types.BIGINT; } else if (fieldType.equals(Byte.class.getName())) { tmpFieldsType[i] = Types.TINYINT; @@ -167,6 +170,8 @@ protected void buildSqlTypes(List fieldTypeArray) { tmpFieldsType[i] = Types.TIMESTAMP; } else if (fieldType.equals(BigDecimal.class.getName())) { tmpFieldsType[i] = Types.DECIMAL; + } else if (fieldType.equals(Date.class.getName())) { + tmpFieldsType[i] = Types.DATE; } else { throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index f170b88e1..9c2bbc9e1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -29,8 +29,6 @@ */ public class RdbTableInfo extends TargetTableInfo { - //private static final String CURR_TYPE = "mysql"; - public static final String URL_KEY = "url"; public static final String TABLE_NAME_KEY = "tableName"; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java deleted file mode 100644 index 7708ba163..000000000 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java +++ /dev/null @@ -1,103 +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.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; -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 1e266e3bc..2bb53064e 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.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; @@ -41,7 +42,7 @@ public String getDriverName() { @Override public RetractJDBCOutputFormat getOutputFormat() { - return new SqlserverOutputFormat(); + return new ExtendOutputFormat(); } @Override From 930e9195cf82d43cdff495bcc64bbc0a6bfa0e14 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:39:04 +0800 Subject: [PATCH 13/78] add extendOutputFormat --- .../sink/rdb/format/ExtendOutputFormat.java | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java new file mode 100644 index 000000000..3aa5e6f99 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -0,0 +1,103 @@ +/* + * 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 ExtendOutputFormat 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 { + 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()); + } + } + } + + +} From 302670a51a9e1849a35a0387fa575cfe1a192cf0 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 5 Dec 2018 11:39:05 +0800 Subject: [PATCH 14/78] ignore case --- .../flink/sql/sink/redis/table/RedisSinkParser.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 5754bdfee..c114b946e 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 @@ -33,15 +33,15 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 5 Dec 2018 15:41:49 +0800 Subject: [PATCH 15/78] modify MathUtil date parse --- .../dtstack/flink/sql/side/rdb/util/MathUtil.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index fd6af5e0b..284d458b3 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -223,10 +223,15 @@ public static Date getDate(Object obj) { return null; } if (obj instanceof String) { - return Date.valueOf((String) obj); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + try { + return new Date(format.parse((String) obj).getTime()); + } catch (ParseException e) { + throw new RuntimeException("String convert to Date fail."); + } } else if (obj instanceof Timestamp) { return new Date(((Timestamp) obj).getTime()); - }else if (obj instanceof Date){ + } else if (obj instanceof Date) { return (Date) obj; } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); @@ -239,9 +244,9 @@ public static Timestamp getTimestamp(Object obj) { if (obj instanceof Timestamp) { return (Timestamp) obj; } else if (obj instanceof Date) { - return Timestamp.valueOf((String) obj); + return new Timestamp(((Date) obj).getTime()); } else if (obj instanceof String) { - return Timestamp.valueOf((String) obj); + return new Timestamp(getDate(obj).getTime()); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } From a455b675f67b2f74e4a381976d20833a141079d1 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Wed, 5 Dec 2018 18:52:20 +0800 Subject: [PATCH 16/78] add cassandra side and sink --- .../cassandra-side/cassandra-all-side/pom.xml | 88 ++++ .../side/cassandra/CassandraAllReqRow.java | 305 ++++++++++++++ .../side/cassandra/CassandraAllSideInfo.java | 113 ++++++ .../cassandra-async-side/pom.xml | 103 +++++ .../side/cassandra/CassandraAsyncReqRow.java | 309 ++++++++++++++ .../cassandra/CassandraAsyncSideInfo.java | 127 ++++++ .../cassandra-side-core/pom.xml | 24 ++ .../cassandra/table/CassandraSideParser.java | 99 +++++ .../table/CassandraSideTableInfo.java | 181 +++++++++ cassandra/cassandra-side/pom.xml | 23 ++ cassandra/cassandra-sink/pom.xml | 82 ++++ .../sink/cassandra/CassandraOutputFormat.java | 379 ++++++++++++++++++ .../sql/sink/cassandra/CassandraSink.java | 133 ++++++ .../cassandra/table/CassandraSinkParser.java | 84 ++++ .../cassandra/table/CassandraTableInfo.java | 161 ++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++ cassandra/pom.xml | 39 ++ docs/cassandraSide.md | 85 ++++ docs/cassandraSink.md | 63 +++ .../flink/sql/side/mongo/MongoAllReqRow.java | 1 - pom.xml | 1 + 21 files changed, 2457 insertions(+), 1 deletion(-) create mode 100644 cassandra/cassandra-side/cassandra-all-side/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java create mode 100644 cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java create mode 100644 cassandra/cassandra-side/cassandra-async-side/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java create mode 100644 cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java create mode 100644 cassandra/cassandra-side/cassandra-side-core/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java create mode 100644 cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java create mode 100644 cassandra/cassandra-side/pom.xml create mode 100644 cassandra/cassandra-sink/pom.xml create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java create mode 100644 cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 cassandra/pom.xml create mode 100644 docs/cassandraSide.md create mode 100644 docs/cassandraSink.md diff --git a/cassandra/cassandra-side/cassandra-all-side/pom.xml b/cassandra/cassandra-side/cassandra-all-side/pom.xml new file mode 100644 index 000000000..74c62afdb --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/pom.xml @@ -0,0 +1,88 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.cassandra + cassandra-all-side + + jar + + + + com.dtstack.flink + sql.side.cassandra.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/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java new file mode 100644 index 000000000..e8ccc739f --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -0,0 +1,305 @@ +/* + * 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.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +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.cassandra.table.CassandraSideTableInfo; +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.net.InetAddress; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAllReqRow extends AllReqRow { + + private static final long serialVersionUID = 54015343561288219L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraAllReqRow.class); + + private static final String cassandra_DRIVER = "com.cassandra.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private transient Cluster cluster; + private transient Session session = null; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new com.dtstack.flink.sql.side.cassandra.CassandraAllSideInfo(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("----- cassandra 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 Session getConn(CassandraSideTableInfo tableInfo) { + try { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = tableInfo.getMaxRequestsPerConnection() == null ? 1 : tableInfo.getMaxRequestsPerConnection(); + Integer coreConnectionsPerHost = tableInfo.getCoreConnectionsPerHost() == null ? 8 : tableInfo.getCoreConnectionsPerHost(); + Integer maxConnectionsPerHost = tableInfo.getMaxConnectionsPerHost() == null ? 32768 : tableInfo.getMaxConnectionsPerHost(); + Integer maxQueueSize = tableInfo.getMaxQueueSize() == null ? 100000 : tableInfo.getMaxQueueSize(); + Integer readTimeoutMillis = tableInfo.getReadTimeoutMillis() == null ? 60000 : tableInfo.getReadTimeoutMillis(); + Integer connectTimeoutMillis = tableInfo.getConnectTimeoutMillis() == null ? 60000 : tableInfo.getConnectTimeoutMillis(); + Integer poolTimeoutMillis = tableInfo.getPoolTimeoutMillis() == null ? 60000 : tableInfo.getPoolTimeoutMillis(); + Integer cassandraPort = 0; + String address = tableInfo.getAddress(); + String userName = tableInfo.getUserName(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connect(database); + LOG.info("connect cassandra is successed!"); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + return session; + } + + + private void loadData(Map>> tmpCache) throws SQLException { + CassandraSideTableInfo tableInfo = (CassandraSideTableInfo) sideInfo.getSideTableInfo(); + Session session = null; + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + try { + session = getConn(tableInfo); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + try { + String connInfo = "address:" + tableInfo.getAddress() + ";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() + " limit " + FETCH_SIZE; + ResultSet resultSet = session.execute(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + for (com.datastax.driver.core.Row row : resultSet) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), row.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 { + try { + if (session != null) { + session.close(); + } + } catch (Exception e) { + LOG.error("Error while closing session.", e); + } + try { + if (cluster != null) { + cluster.close(); + } + } catch (Exception e) { + LOG.error("Error while closing cluster.", e); + } + } + } +} diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java new file mode 100644 index 000000000..bba39fc3e --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -0,0 +1,113 @@ +/* + * 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.cassandra; + +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.cassandra.table.CassandraSideTableInfo; +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/22 + * + * @author xuqianjin + */ +public class CassandraAllSideInfo extends SideInfo { + + private static final long serialVersionUID = -8690814317653033557L; + + public CassandraAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase() + "." + + cassandraSideTableInfo.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/cassandra/cassandra-side/cassandra-async-side/pom.xml b/cassandra/cassandra-side/cassandra-async-side/pom.xml new file mode 100644 index 000000000..cd709fecd --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/pom.xml @@ -0,0 +1,103 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.async.cassandra + + cassandra-async-side + + jar + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + com.dtstack.flink + sql.side.cassandra.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/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java new file mode 100644 index 000000000..94c8e6fb6 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -0,0 +1,309 @@ +/* + * 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.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +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.cassandra.table.CassandraSideTableInfo; +import com.google.common.base.Function; +import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.vertx.core.json.JsonArray; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +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.net.InetAddress; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 6631584128079864735L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraAsyncReqRow.class); + + 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; + + private transient Cluster cluster; + private transient ListenableFuture session; + private transient CassandraSideTableInfo cassandraSideTableInfo; + + public CassandraAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new com.dtstack.flink.sql.side.cassandra.CassandraAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + cassandraSideTableInfo = (CassandraSideTableInfo) sideInfo.getSideTableInfo(); + connCassandraDB(cassandraSideTableInfo); + } + + private void connCassandraDB(CassandraSideTableInfo tableInfo) { + try { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = tableInfo.getMaxRequestsPerConnection() == null ? 1 : tableInfo.getMaxRequestsPerConnection(); + Integer coreConnectionsPerHost = tableInfo.getCoreConnectionsPerHost() == null ? 8 : tableInfo.getCoreConnectionsPerHost(); + Integer maxConnectionsPerHost = tableInfo.getMaxConnectionsPerHost() == null ? 32768 : tableInfo.getMaxConnectionsPerHost(); + Integer maxQueueSize = tableInfo.getMaxQueueSize() == null ? 100000 : tableInfo.getMaxQueueSize(); + Integer readTimeoutMillis = tableInfo.getReadTimeoutMillis() == null ? 60000 : tableInfo.getReadTimeoutMillis(); + Integer connectTimeoutMillis = tableInfo.getConnectTimeoutMillis() == null ? 60000 : tableInfo.getConnectTimeoutMillis(); + Integer poolTimeoutMillis = tableInfo.getPoolTimeoutMillis() == null ? 60000 : tableInfo.getPoolTimeoutMillis(); + Integer cassandraPort = 0; + String address = tableInfo.getAddress(); + String userName = tableInfo.getUserName(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connectAsync(database); + LOG.info("connect cassandra is successed!"); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + StringBuffer stringBuffer = new StringBuffer(); + String sqlWhere = " where "; + + for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = input.getField(conValIndex); + if (equalObj == null) { + resultFuture.complete(null); + } + inputParams.add(equalObj); + stringBuffer.append(sideInfo.getEqualFieldList().get(i)) + .append(" = ").append("'" + equalObj + "'") + .append(" and "); + } + + String key = buildCacheKey(inputParams); + sqlWhere = sqlWhere + stringBuffer.toString().substring(0, stringBuffer.lastIndexOf(" and ")); + + 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 rowArray : (List) val.getContent()) { + Row row = fillData(input, rowArray); + resultFuture.complete(Collections.singleton(row)); + } + + } else { + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + //connect Cassandra + connCassandraDB(cassandraSideTableInfo); + + String sqlCondition = sideInfo.getSqlCondition() + " " + sqlWhere; + System.out.println("sqlCondition:" + sqlCondition); + + ListenableFuture resultSet = Futures.transformAsync(session, + new AsyncFunction() { + @Override + public ListenableFuture apply(Session session) throws Exception { + return session.executeAsync(sqlCondition); + } + }); + + ListenableFuture> data = Futures.transform(resultSet, + new Function>() { + @Override + public List apply(ResultSet rs) { + return rs.all(); + } + }); + + Futures.addCallback(data, new FutureCallback>() { + @Override + public void onSuccess(List rows) { + cluster.closeAsync(); + if (rows.size() > 0) { + List cacheContent = Lists.newArrayList(); + for (com.datastax.driver.core.Row line : rows) { + 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()); + } + resultFuture.complete(null); + } + } + + @Override + public void onFailure(Throwable t) { + LOG.error("Failed to retrieve the data: %s%n", + t.getMessage()); + System.out.println("Failed to retrieve the data: " + t.getMessage()); + cluster.closeAsync(); + resultFuture.complete(null); + } + }); + } + + @Override + public Row fillData(Row input, Object line) { + com.datastax.driver.core.Row rowArray = (com.datastax.driver.core.Row) 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 (rowArray == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), rowArray.getObject(entry.getValue())); + } + } + + System.out.println("row:" + row.toString()); + return row; + } + + @Override + public void close() throws Exception { + super.close(); + if (cluster != null) { + cluster.close(); + cluster = null; + } + } + + public String buildCacheKey(JsonArray jsonArray) { + StringBuilder sb = new StringBuilder(); + for (Object ele : jsonArray.getList()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } +} diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java new file mode 100644 index 000000000..b1d239440 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java @@ -0,0 +1,127 @@ +/* + * 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.cassandra; + +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.cassandra.table.CassandraSideTableInfo; +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/22 + * + * @author xuqianjin + */ +public class CassandraAsyncSideInfo extends SideInfo { + + private static final long serialVersionUID = -4403313049809013362L; + + public CassandraAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) 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}"; + + sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase()+"."+cassandraSideTableInfo.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/cassandra/cassandra-side/cassandra-side-core/pom.xml b/cassandra/cassandra-side/cassandra-side-core/pom.xml new file mode 100644 index 000000000..a3137b763 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/pom.xml @@ -0,0 +1,24 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.cassandra.core + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar + + \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java new file mode 100644 index 000000000..6403a225b --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java @@ -0,0 +1,99 @@ +/* + * 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.cassandra.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; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSideParser 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$"); + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, CassandraSideParser::dealSideSign); + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo cassandraSideTableInfo = new com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo(); + cassandraSideTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, cassandraSideTableInfo); + parseCacheProp(cassandraSideTableInfo, props); + + cassandraSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + cassandraSideTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + cassandraSideTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + cassandraSideTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + cassandraSideTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + cassandraSideTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxRequestsPerConnection(MathUtil.getIntegerVal(props.get(MAX_REQUEST_PER_CONNECTION_KEY.toLowerCase()))); + cassandraSideTableInfo.setCoreConnectionsPerHost(MathUtil.getIntegerVal(props.get(CORE_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxConnectionsPerHost(MathUtil.getIntegerVal(props.get(MAX_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxQueueSize(MathUtil.getIntegerVal(props.get(MAX_QUEUE_SIZE_KEY.toLowerCase()))); + cassandraSideTableInfo.setReadTimeoutMillis(MathUtil.getIntegerVal(props.get(READ_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraSideTableInfo.setConnectTimeoutMillis(MathUtil.getIntegerVal(props.get(CONNECT_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraSideTableInfo.setPoolTimeoutMillis(MathUtil.getIntegerVal(props.get(POOL_TIMEOUT_MILLIS_KEY.toLowerCase()))); + + return cassandraSideTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo) { + } +} diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java new file mode 100644 index 000000000..b1b36f7e8 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java @@ -0,0 +1,181 @@ +/* + * 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.cassandra.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -5556431094535478915L; + + private static final String CURR_TYPE = "cassandra"; + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + 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; + } + + public Integer getMaxRequestsPerConnection() { + return maxRequestsPerConnection; + } + + public void setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + this.maxRequestsPerConnection = maxRequestsPerConnection; + } + + public Integer getCoreConnectionsPerHost() { + return coreConnectionsPerHost; + } + + public void setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + this.coreConnectionsPerHost = coreConnectionsPerHost; + } + + public Integer getMaxConnectionsPerHost() { + return maxConnectionsPerHost; + } + + public void setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + this.maxConnectionsPerHost = maxConnectionsPerHost; + } + + public Integer getMaxQueueSize() { + return maxQueueSize; + } + + public void setMaxQueueSize(Integer maxQueueSize) { + this.maxQueueSize = maxQueueSize; + } + + public Integer getReadTimeoutMillis() { + return readTimeoutMillis; + } + + public void setReadTimeoutMillis(Integer readTimeoutMillis) { + this.readTimeoutMillis = readTimeoutMillis; + } + + public Integer getConnectTimeoutMillis() { + return connectTimeoutMillis; + } + + public void setConnectTimeoutMillis(Integer connectTimeoutMillis) { + this.connectTimeoutMillis = connectTimeoutMillis; + } + + public Integer getPoolTimeoutMillis() { + return poolTimeoutMillis; + } + + public void setPoolTimeoutMillis(Integer poolTimeoutMillis) { + this.poolTimeoutMillis = poolTimeoutMillis; + } + + public CassandraSideTableInfo() { + setType(CURR_TYPE); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "Cassandra field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Cassandra field of database is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } +} diff --git a/cassandra/cassandra-side/pom.xml b/cassandra/cassandra-side/pom.xml new file mode 100644 index 000000000..92d058900 --- /dev/null +++ b/cassandra/cassandra-side/pom.xml @@ -0,0 +1,23 @@ + + + + sql.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.cassandra + cassandra-side + + cassandra-side-core + cassandra-async-side + cassandra-all-side + + + pom + + \ No newline at end of file diff --git a/cassandra/cassandra-sink/pom.xml b/cassandra/cassandra-sink/pom.xml new file mode 100644 index 000000000..4fb20c373 --- /dev/null +++ b/cassandra/cassandra-sink/pom.xml @@ -0,0 +1,82 @@ + + + sql.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.cassandra + jar + + cassandra-sink + http://maven.apache.org + + + + + + + + + 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 + + + + + + + + + + + + + + + + + + diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java new file mode 100644 index 000000000..11fe24b81 --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java @@ -0,0 +1,379 @@ +/* + * 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. + */ + + + +/* + * 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.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +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.net.InetAddress; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.util.ArrayList; + +/** + * OutputFormat to write tuples into a database. + * The OutputFormat has to be configured using the supplied OutputFormatBuilder. + * + * @see Tuple + * @see DriverManager + */ +public class CassandraOutputFormat extends RichOutputFormat { + private static final long serialVersionUID = -7994311331389155692L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class); + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + private int batchInterval = 5000; + + private Cluster cluster; + private Session session = null; + + private int batchCount = 0; + + private transient Counter outRecords; + + private transient Meter outRecordsRate; + + public CassandraOutputFormat() { + } + + @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 { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = this.maxRequestsPerConnection == null ? 1 : this.maxRequestsPerConnection; + Integer coreConnectionsPerHost = this.coreConnectionsPerHost == null ? 8 : this.coreConnectionsPerHost; + Integer maxConnectionsPerHost = this.maxConnectionsPerHost == null ? 32768 : this.maxConnectionsPerHost; + Integer maxQueueSize = this.maxQueueSize == null ? 100000 : this.maxQueueSize; + Integer readTimeoutMillis = this.readTimeoutMillis == null ? 60000 : this.readTimeoutMillis; + Integer connectTimeoutMillis = this.connectTimeoutMillis == null ? 60000 : this.connectTimeoutMillis; + Integer poolTimeoutMillis = this.poolTimeoutMillis == null ? 60000 : this.poolTimeoutMillis; + Integer cassandraPort = 0; + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connect(database); + LOG.info("connect cassandra is successed!"); + initMetric(); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + } + + 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)); + } + + /** + * 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); + try { + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing + } + } catch (Exception e) { + throw new IllegalArgumentException("writeRecord() failed", e); + } + } + + private void insertWrite(Row row) { + try { + String cql = buildSql(row); + if (cql != null) { + ResultSet resultSet = session.execute(cql); + resultSet.wasApplied(); + } + } catch (Exception e) { + LOG.error("[upsert] is error:" + e.getMessage()); + } + } + + private String buildSql(Row row) { + StringBuffer fields = new StringBuffer(); + StringBuffer values = new StringBuffer(); + for (int index = 0; index < row.getArity(); index++) { + if (row.getField(index) == null) { + } else { + fields.append(fieldNames[index] + ","); + values.append("'" + row.getField(index) + "'" + ","); + } + } + fields.deleteCharAt(fields.length() - 1); + values.deleteCharAt(values.length() - 1); + String cql = "INSERT INTO " + database + "." + tableName + " (" + fields.toString() + ") " + + " VALUES (" + values.toString() + ")"; + return cql; + } + + /** + * 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 (session != null) { + session.close(); + } + } catch (Exception e) { + LOG.error("Error while closing session.", e); + } + try { + if (cluster != null) { + cluster.close(); + } + } catch (Exception e) { + LOG.error("Error while closing cluster.", e); + } + LOG.info("close cassandra is successed!"); + } + + public static CassandraFormatBuilder buildOutputFormat() { + return new CassandraFormatBuilder(); + } + + public static class CassandraFormatBuilder { + private final CassandraOutputFormat format; + + protected CassandraFormatBuilder() { + this.format = new CassandraOutputFormat(); + } + + public CassandraFormatBuilder setUsername(String username) { + format.userName = username; + return this; + } + + public CassandraFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public CassandraFormatBuilder setAddress(String address) { + format.address = address; + return this; + } + + public CassandraFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + public CassandraFormatBuilder setDatabase(String database) { + format.database = database; + return this; + } + + public CassandraFormatBuilder setFieldNames(String[] fieldNames) { + format.fieldNames = fieldNames; + return this; + } + + public CassandraFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + format.fieldTypes = fieldTypes; + return this; + } + + public CassandraFormatBuilder setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + format.maxRequestsPerConnection = maxRequestsPerConnection; + return this; + } + + public CassandraFormatBuilder setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + format.coreConnectionsPerHost = coreConnectionsPerHost; + return this; + } + + public CassandraFormatBuilder setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + format.maxConnectionsPerHost = maxConnectionsPerHost; + return this; + } + + public CassandraFormatBuilder setMaxQueueSize(Integer maxQueueSize) { + format.maxQueueSize = maxQueueSize; + return this; + } + + public CassandraFormatBuilder setReadTimeoutMillis(Integer readTimeoutMillis) { + format.readTimeoutMillis = readTimeoutMillis; + return this; + } + + public CassandraFormatBuilder setConnectTimeoutMillis(Integer connectTimeoutMillis) { + format.connectTimeoutMillis = connectTimeoutMillis; + return this; + } + + public CassandraFormatBuilder setPoolTimeoutMillis(Integer poolTimeoutMillis) { + format.poolTimeoutMillis = poolTimeoutMillis; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public CassandraOutputFormat 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.address == null) { + throw new IllegalArgumentException("No address URL supplied."); + } + if (format.database == null) { + throw new IllegalArgumentException("No dababase suplied"); + } + if (format.tableName == null) { + throw new IllegalArgumentException("No tableName supplied"); + } + return format; + } + } +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java new file mode 100644 index 000000000..eb7b23b53 --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java @@ -0,0 +1,133 @@ +/* + * 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.cassandra; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.cassandra.table.CassandraTableInfo; +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.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; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSink implements RetractStreamTableSink, IStreamSinkGener { + + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + protected String address; + protected String tableName; + protected String userName; + protected String password; + protected String database; + protected Integer maxRequestsPerConnection; + protected Integer coreConnectionsPerHost; + protected Integer maxConnectionsPerHost; + protected Integer maxQueueSize; + protected Integer readTimeoutMillis; + protected Integer connectTimeoutMillis; + protected Integer poolTimeoutMillis; + + public CassandraSink() { + // TO DO NOTHING + } + + @Override + public CassandraSink genStreamSink(TargetTableInfo targetTableInfo) { + CassandraTableInfo cassandraTableInfo = (CassandraTableInfo) targetTableInfo; + this.address = cassandraTableInfo.getAddress(); + this.tableName = cassandraTableInfo.getTableName(); + this.userName = cassandraTableInfo.getUserName(); + this.password = cassandraTableInfo.getPassword(); + this.database = cassandraTableInfo.getDatabase(); + this.maxRequestsPerConnection = cassandraTableInfo.getMaxRequestsPerConnection(); + this.coreConnectionsPerHost = cassandraTableInfo.getCoreConnectionsPerHost(); + this.maxConnectionsPerHost = cassandraTableInfo.getMaxConnectionsPerHost(); + this.maxQueueSize = cassandraTableInfo.getMaxQueueSize(); + this.readTimeoutMillis = cassandraTableInfo.getReadTimeoutMillis(); + this.connectTimeoutMillis = cassandraTableInfo.getConnectTimeoutMillis(); + this.poolTimeoutMillis = cassandraTableInfo.getPoolTimeoutMillis(); + return this; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + CassandraOutputFormat.CassandraFormatBuilder builder = CassandraOutputFormat.buildOutputFormat(); + builder.setAddress(this.address) + .setDatabase(this.database) + .setTableName(this.tableName) + .setPassword(this.password) + .setUsername(this.userName) + .setMaxRequestsPerConnection(this.maxRequestsPerConnection) + .setCoreConnectionsPerHost(this.coreConnectionsPerHost) + .setMaxConnectionsPerHost(this.maxConnectionsPerHost) + .setMaxQueueSize(this.maxQueueSize) + .setReadTimeoutMillis(this.readTimeoutMillis) + .setConnectTimeoutMillis(this.connectTimeoutMillis) + .setPoolTimeoutMillis(this.poolTimeoutMillis) + .setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + + CassandraOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @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; + } + +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java new file mode 100644 index 000000000..4c68e71ae --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java @@ -0,0 +1,84 @@ +/* + * 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.cassandra.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; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSinkParser extends AbsTableParser { + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + CassandraTableInfo cassandraTableInfo = new CassandraTableInfo(); + cassandraTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, cassandraTableInfo); + + cassandraTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + cassandraTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + cassandraTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + cassandraTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + cassandraTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + cassandraTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + cassandraTableInfo.setMaxRequestsPerConnection(MathUtil.getIntegerVal(props.get(MAX_REQUEST_PER_CONNECTION_KEY.toLowerCase()))); + cassandraTableInfo.setCoreConnectionsPerHost(MathUtil.getIntegerVal(props.get(CORE_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraTableInfo.setMaxConnectionsPerHost(MathUtil.getIntegerVal(props.get(MAX_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraTableInfo.setMaxQueueSize(MathUtil.getIntegerVal(props.get(MAX_QUEUE_SIZE_KEY.toLowerCase()))); + cassandraTableInfo.setReadTimeoutMillis(MathUtil.getIntegerVal(props.get(READ_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraTableInfo.setConnectTimeoutMillis(MathUtil.getIntegerVal(props.get(CONNECT_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraTableInfo.setPoolTimeoutMillis(MathUtil.getIntegerVal(props.get(POOL_TIMEOUT_MILLIS_KEY.toLowerCase()))); + + return cassandraTableInfo; + } +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java new file mode 100644 index 000000000..7d52b23bb --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java @@ -0,0 +1,161 @@ +/* + * 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.cassandra.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "cassandra"; + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + public CassandraTableInfo() { + setType(CURR_TYPE); + } + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + 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; + } + + public Integer getMaxRequestsPerConnection() { + return maxRequestsPerConnection; + } + + public void setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + this.maxRequestsPerConnection = maxRequestsPerConnection; + } + + public Integer getCoreConnectionsPerHost() { + return coreConnectionsPerHost; + } + + public void setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + this.coreConnectionsPerHost = coreConnectionsPerHost; + } + + public Integer getMaxConnectionsPerHost() { + return maxConnectionsPerHost; + } + + public void setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + this.maxConnectionsPerHost = maxConnectionsPerHost; + } + + public Integer getMaxQueueSize() { + return maxQueueSize; + } + + public void setMaxQueueSize(Integer maxQueueSize) { + this.maxQueueSize = maxQueueSize; + } + + public Integer getReadTimeoutMillis() { + return readTimeoutMillis; + } + + public void setReadTimeoutMillis(Integer readTimeoutMillis) { + this.readTimeoutMillis = readTimeoutMillis; + } + + public Integer getConnectTimeoutMillis() { + return connectTimeoutMillis; + } + + public void setConnectTimeoutMillis(Integer connectTimeoutMillis) { + this.connectTimeoutMillis = connectTimeoutMillis; + } + + public Integer getPoolTimeoutMillis() { + return poolTimeoutMillis; + } + + public void setPoolTimeoutMillis(Integer poolTimeoutMillis) { + this.poolTimeoutMillis = poolTimeoutMillis; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "Cassandra field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Cassandra field of database is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.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.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/cassandra/pom.xml b/cassandra/pom.xml new file mode 100644 index 000000000..f49de388b --- /dev/null +++ b/cassandra/pom.xml @@ -0,0 +1,39 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.cassandra + pom + + + cassandra-sink + cassandra-side + + + + + junit + junit + 3.8.1 + test + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + com.datastax.cassandra + cassandra-driver-core + 3.6.0 + + + + \ No newline at end of file diff --git a/docs/cassandraSide.md b/docs/cassandraSide.md new file mode 100644 index 000000000..131560047 --- /dev/null +++ b/docs/cassandraSide.md @@ -0,0 +1,85 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='cassandra', + address ='ip:port[,ip:port]', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + database='database', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + cassandra-3.6.x + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type |表明 输出表类型 cassandra|是|| + | address | 连接cassandra数据库 jdbcUrl |是|| + | userName | cassandra连接用户名|否|| + | password | cassandra连接密码|否|| + | tableName | cassandra表名称|是|| + | database | cassandra表名称|是|| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| + | coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| + | maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| + | maxQueueSize | Cassandra队列大小|否|NONE| + | readTimeoutMillis | Cassandra读超时|否|NONE| + | connectTimeoutMillis | Cassandra连接超时|否|NONE| + | poolTimeoutMillis | Cassandra线程池超时|否|NONE| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + + +## 5.样例 +``` +create table sideTable( + CHANNEL varchar, + XCCOUNT int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='cassandra', + address ='172.21.32.1:9042,172.21.32.1:9042', + database ='test', + tableName ='sidetest', + cache ='LRU', + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + diff --git a/docs/cassandraSink.md b/docs/cassandraSink.md new file mode 100644 index 000000000..8ea38e104 --- /dev/null +++ b/docs/cassandraSink.md @@ -0,0 +1,63 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='cassandra', + address ='ip:port[,ip:port]', + userName ='userName', + password ='pwd', + database ='databaseName', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + cassandra-3.6.x + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型 cassandra|是|| +|address | 连接cassandra数据库 jdbcUrl |是|| +|userName | cassandra连接用户名|否|| +|password | cassandra连接密码|否|| +|tableName | cassandra表名称|是|| +|database | cassandra表名称|是|| +|parallelism | 并行度设置|否|1| +|maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| +|coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| +|maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| +|maxQueueSize | Cassandra队列大小|否|NONE| +|readTimeoutMillis | Cassandra读超时|否|NONE| +|connectTimeoutMillis | Cassandra连接超时|否|NONE| +|poolTimeoutMillis | Cassandra线程池超时|否|NONE| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='cassandra', + address ='172.21.32.1:9042,172.21.32.1:9042', + userName ='dtstack', + password ='abc123', + database ='test', + tableName ='pv', + parallelism ='1' + ) + ``` \ No newline at end of file diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index e1a398d4c..cb142944d 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -152,7 +152,6 @@ public void flatMap(Row value, Collector out) throws Exception { } for (Map one : cacheList) { - System.out.println(fillData(value, one)); out.collect(fillData(value, one)); } } diff --git a/pom.xml b/pom.xml index 07694589c..837ade4b2 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,7 @@ mongo redis5 launcher + cassandra pom From 3226f6fc25498268423861b9137975386390a8ed Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 5 Dec 2018 20:50:10 +0800 Subject: [PATCH 17/78] bugfix --- .../dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index e8ccc739f..3a7e56902 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -81,7 +81,7 @@ public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { From d91101ca514af2e6c57967d0f3ff2b6677a2beaf Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 5 Dec 2018 20:53:51 +0800 Subject: [PATCH 18/78] bugfix --- .../dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index e8ccc739f..3a7e56902 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -81,7 +81,7 @@ public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { From b382bbe2efb67f8ccf8111c0d7f208add2eb6352 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 6 Dec 2018 09:54:18 +0800 Subject: [PATCH 19/78] add cassandra md --- README.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 27dffc8dc..f7c86bfab 100644 --- a/README.md +++ b/README.md @@ -8,8 +8,8 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase,mongo,redis - * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis + * 维表:mysql,hbase,mongo,redis,cassandra + * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 * 增加oracle维表,结果表功能 @@ -149,12 +149,14 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mysql 结果表插件](docs/mysqlSink.md) * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) +* [cassandra 结果表插件](docs/cassandraSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) * [mysql 维表插件](docs/mysqlSide.md) * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) +* [cassandra 维表插件](docs/cassandraSide.md) ## 3 性能指标(新增) From 765bff604ed014ff0d6932f6e08e637605658751 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 6 Dec 2018 10:24:58 +0800 Subject: [PATCH 20/78] Update README.md --- README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index fc9f278c6..ad14222ca 100644 --- a/README.md +++ b/README.md @@ -10,12 +10,10 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase,mongo,redis,cassandra - * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis,cassandra + * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra + * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 - * 增加oracle维表,结果表功能 - * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 From 88c701aeff9fe093fafdab6a9e9ca485c3b8b9f0 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 6 Dec 2018 14:22:29 +0800 Subject: [PATCH 21/78] fix flink-1.6 local-mode run bug --- .../dtstack/flink/sql/MyLocalStreamEnvironment.java | 4 ++-- .../flink/sql/launcher/LauncherOptionParser.java | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java index f41ecf00b..acd28e3f3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java @@ -100,8 +100,8 @@ public JobExecutionResult execute(String jobName) throws Exception { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "-1L"); - configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); + configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "512M"); + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined configuration.addAll(this.conf); 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 75c5c4f0f..a801b193e 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 @@ -101,21 +101,25 @@ public LauncherOptionParser(String[] args) { byte[] filecontent = new byte[(int) file.length()]; in.read(filecontent); String content = new String(filecontent, "UTF-8"); + String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); properties.setSql(sql); + String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); properties.setLocalSqlPluginPath(localPlugin); + String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - if(!ClusterMode.local.name().equals(mode)){ - Preconditions.checkNotNull(remotePlugin); - properties.setRemoteSqlPluginPath(remotePlugin); - } + Preconditions.checkNotNull(remotePlugin); + properties.setRemoteSqlPluginPath(remotePlugin); + String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); properties.setName(name); + String addJar = cl.getOptionValue(OPTION_ADDJAR); if(StringUtils.isNotBlank(addJar)){ properties.setAddjar(addJar); } + String confProp = cl.getOptionValue(OPTION_CONF_PROP); if(StringUtils.isNotBlank(confProp)){ properties.setConfProp(confProp); @@ -157,6 +161,7 @@ public LauncherOptions getLauncherOptions(){ public List getProgramExeArgList() throws Exception { Map mapConf = PluginUtil.ObjectToMap(properties); List args = Lists.newArrayList(); + for(Map.Entry one : mapConf.entrySet()){ String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) From b41cc0ea7f3eb01dfaef5c8cf6ed6a67e6c2ec78 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 10 Dec 2018 11:36:58 +0800 Subject: [PATCH 22/78] password is null --- .../java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 2 ++ redis5/redis5-sink/pom.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) 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 3f9f40f24..1a6aa0425 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 @@ -80,6 +80,8 @@ private void buildRedisClient(RedisSideTableInfo tableInfo){ String password = redisSideTableInfo.getPassword(); if (password != null){ password = password + "@"; + } else { + password = ""; } String database = redisSideTableInfo.getDatabase(); if (database == null){ diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml index 994c3972a..2919d8685 100644 --- a/redis5/redis5-sink/pom.xml +++ b/redis5/redis5-sink/pom.xml @@ -26,7 +26,7 @@ redis.clients jedis - 2.8.0 + 2.9.0 From 43df5edc0edecaeb20c3ecc9bda9edf77df245f0 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Mon, 10 Dec 2018 16:51:06 +0800 Subject: [PATCH 23/78] fix kafka offset reset option --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ 6 files changed, 12 insertions(+) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 898083826..606d42c9d 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -46,6 +46,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 10 Dec 2018 17:05:34 +0800 Subject: [PATCH 24/78] fix bug for kafka offset --- .../flink/sql/source/kafka/table/KafkaSourceParser.java | 3 ++- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceParser.java | 1 + .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 3 ++- .../flink/sql/source/kafka/table/KafkaSourceParser.java | 1 + .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ 6 files changed, 10 insertions(+), 2 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 898083826..301b0b423 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.source.kafka.table; @@ -46,6 +46,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 10 Dec 2018 18:12:36 +0800 Subject: [PATCH 25/78] =?UTF-8?q?fix=20up=20=E7=94=A8=E6=88=B7=E6=A0=87?= =?UTF-8?q?=E8=AF=86=E7=AC=A6SqlParse=E5=BC=BA=E5=88=B6=E4=B8=BA=E5=A4=A7?= =?UTF-8?q?=E5=86=99=EF=BC=8C=E5=BA=94=E8=AF=A5=E4=BF=9D=E6=8C=81=E7=94=A8?= =?UTF-8?q?=E6=88=B7=E8=BE=93=E5=85=A5=E4=B8=8D=E5=8F=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 9 +++++++-- .../dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../flink/sql/parser/CreateTmpTableParser.java | 12 +++++++++--- .../dtstack/flink/sql/parser/InsertSqlParser.java | 7 ++++++- .../com/dtstack/flink/sql/side/SideSQLParser.java | 8 ++++++-- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- .../com/dtstack/flink/sql/table/AbsTableParser.java | 3 --- 7 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 610abf21f..2e9c979d1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -34,6 +34,7 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -185,7 +186,11 @@ public static void main(String[] args) throws Exception { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); - SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); + org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); @@ -246,7 +251,7 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); } classLoader.loadClass(funcInfo.getClassName()); - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName().toUpperCase(), + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index d10d6825e..b5e4a4aa4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -54,7 +54,7 @@ public boolean verify(String sql) { public void parseSql(String sql, SqlTree sqlTree) { Matcher matcher = PATTERN.matcher(sql); if(matcher.find()){ - String tableName = matcher.group(1).toUpperCase(); + String tableName = matcher.group(1); String fieldsInfoStr = matcher.group(2); String propsStr = matcher.group(3); Map props = parseProp(propsStr); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 8da89b2be..d205c59b6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -62,11 +63,16 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String selectSql = null; if(matcher.find()) { - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); selectSql = "select " + matcher.group(2); } - SqlParser sqlParser = SqlParser.create(selectSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(selectSql,config); + SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); @@ -89,7 +95,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String fieldsInfoStr = null; if (matcher.find()){ - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); fieldsInfoStr = matcher.group(2); } CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e08540c92..7d5c61e7a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlJoin; @@ -57,7 +58,11 @@ public static InsertSqlParser newInstance(){ @Override public void parseSql(String sql, SqlTree sqlTree) { - SqlParser sqlParser = SqlParser.create(sql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 7f165ac30..472146c11 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -52,11 +53,14 @@ public class SideSQLParser { public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - exeSql = DtStringUtil.replaceIgnoreQuota(exeSql, "`", ""); System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); - SqlParser sqlParser = SqlParser.create(exeSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); 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 080d3d48b..120bd4ee6 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 @@ -642,7 +642,7 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab String[] filedNameArr = new String[filed.length - 1]; System.arraycopy(filed, 0, filedNameArr, 0, filed.length - 1); String fieldName = String.join(" ", filedNameArr); - fieldNames.add(fieldName.toUpperCase()); + fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); Class tableField = table.getSchema().getType(i).get().getTypeClass(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 2fc2d799b..c3ccdd789 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -82,9 +82,6 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); - if(fieldNameNeedsUpperCase()) { - fieldRow = fieldRow.toUpperCase(); - } boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); From f8d692354e8b40dcc3bdab580cdd93e935d68db1 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 10 Dec 2018 20:36:55 +0800 Subject: [PATCH 26/78] add default database --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 6 +++++- 1 file changed, 5 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 910882c9f..c2a33255d 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 @@ -108,6 +108,10 @@ private void establishConnection() { if (timeout == 0){ timeout = 10000; } + if (database == null) + { + database = "0"; + } switch (redisType){ //单机 @@ -134,7 +138,7 @@ public void writeRecord(Tuple2 record) throws IOException { return; } Row row = tupleTrans.getField(1); - if (record.getArity() != fieldNames.length) { + if (row.getArity() != fieldNames.length) { return; } From f223993275b3973d4fdf8c3ef2ca1e176c008a06 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 11 Dec 2018 09:34:06 +0800 Subject: [PATCH 27/78] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ad14222ca..5dfa80efb 100644 --- a/README.md +++ b/README.md @@ -188,7 +188,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ``` -CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun +CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun; CREATE TABLE MyTable( From a635352d15fcc5b023f8b058d4957982afcde223 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 14:43:53 +0800 Subject: [PATCH 28/78] add pattern match for kafka topic and fix groupId invalid, check method not called bug --- docs/kafkaSource.md | 7 +- .../source/kafka/CustomerKafka09Consumer.java | 8 +- .../flink/sql/source/kafka/KafkaSource.java | 14 +- .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 138 +++++++++-------- .../kafka/CustomerKafka010Consumer.java | 8 +- .../flink/sql/source/kafka/KafkaSource.java | 102 +++++++------ .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 15 ++ .../kafka/CustomerKafka011Consumer.java | 10 +- .../flink/sql/source/kafka/KafkaSource.java | 113 +++++++------- .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 139 ++++++++++-------- .../sql/side/rdb/table/RdbSideParser.java | 2 +- .../sql/sink/rdb/table/RdbSinkParser.java | 1 + 15 files changed, 337 insertions(+), 226 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index f382ba9ad..f1518aaec 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -38,6 +38,8 @@ CREATE TABLE tableName( |bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| +|topicIsPattern | topic是否是正则表达式格式|否| false +|groupId | 需要读取的 groupId 名称|是|| |offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| @@ -54,7 +56,10 @@ CREATE TABLE MyTable( bootstrapServers ='172.16.8.198:9092', zookeeperQuorum ='172.16.8.198:2181/kafka', offsetReset ='latest', - topic ='nbTest1', + groupId='nbTest', + topic ='nbTest1,nbTest2,nbTest3', + --- topic ='mqTest.*', + ---topicIsPattern='true', parallelism ='1' ); ``` diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java index 052b4f048..09e684c75 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java @@ -30,8 +30,10 @@ import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; +import java.util.Arrays; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * Reason: @@ -47,7 +49,11 @@ public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + public CustomerKafka09Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } 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 cbc697c82..782022270 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 @@ -25,6 +25,7 @@ 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.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -39,6 +40,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime @@ -67,6 +69,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka09SourceTableInfo.getFields().length]; @@ -75,8 +79,14 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); - FlinkKafkaConsumer09 kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + FlinkKafkaConsumer09 kafkaSrc; + if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka09Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 301b0b423..664e8c4b1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + public CustomerKafka010Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } @Override public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); 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 2953d0f86..3fb175804 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 @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka; @@ -25,6 +24,7 @@ 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.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -38,66 +38,78 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSource implements IStreamSourceGener { - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka010SourceTableInfo.getTopic(); + KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka010SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) - Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka010SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); + } - TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; - for(int i = 0; i< kafka010SourceTableInfo.getFieldClasses().length; i++){ - types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); - } + TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); - TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); - FlinkKafkaConsumer010 kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + FlinkKafkaConsumer010 kafkaSrc; + if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka010Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } - //earliest,latest - if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ - kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} - try { - 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(); - } + //earliest,latest + if ("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + try { + 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(); + } - String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); - } + String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 65778d22c..4a2590536 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerKafka011Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } 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 d10151920..5f312fa33 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 @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka; @@ -25,6 +24,7 @@ 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.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -38,66 +38,79 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSource implements IStreamSourceGener
{ - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka011SourceTableInfo.getTopic(); - - Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) - - TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; - for(int i = 0; i< kafka011SourceTableInfo.getFieldClasses().length; i++){ - types[i] = TypeInformation.of(kafka011SourceTableInfo.getFieldClasses()[i]); - } - - TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); - FlinkKafkaConsumer011 kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); - - //earliest,latest - if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ - kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} - try { - 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(); - } - - String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); - } + KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka011SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + + TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka011SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka011SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); + + FlinkKafkaConsumer011 kafkaSrc; + if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka011Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + + + //earliest,latest + if ("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + try { + 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(); + } + + String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index f71d44f74..47d453adc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 11 Dec 2018 16:37:49 +0800 Subject: [PATCH 29/78] add password --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 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 c2a33255d..71607e918 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 @@ -126,7 +126,7 @@ private void establishConnection() { break; //集群 case 3: - jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + jedis = new JedisCluster(addresses, timeout, timeout,10, password, poolConfig); } } From 72a90a92039e24e9f77bb671380435e4c89d8b10 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 17:18:42 +0800 Subject: [PATCH 30/78] remove groupId check --- docs/kafkaSource.md | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index f1518aaec..748948a7b 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -39,7 +39,7 @@ CREATE TABLE tableName( |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| |topicIsPattern | topic是否是正则表达式格式|否| false -|groupId | 需要读取的 groupId 名称|是|| +|groupId | 需要读取的 groupId 名称|否|| |offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index a17848e97..4daf83d6f 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,7 +121,7 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 2d948fa74..93e7d7642 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,7 +121,7 @@ public void setTopicIsPattern(Boolean topicIsPattern) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 36d78bdd8..b151a93e9 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); From d783df65f96f1c3f33029b3da1d0bb27c7245560 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:15:30 +0800 Subject: [PATCH 31/78] groupid check null --- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 4 files changed, 11 insertions(+), 5 deletions(-) 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 782022270..b08485e0b 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 @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka09SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) 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 3fb175804..523eb25dc 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 @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka010SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) 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 5f312fa33..5c9f5eb49 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 @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka011SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..8e513adfc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); + column_name = column_name; } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); + getFullField().add(columnName); } } } From c9ecc83150f531f1d5d3fbdb27b44e5112021223 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:40:19 +0800 Subject: [PATCH 32/78] reset ExtendOutputFormat --- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 8e513adfc..3aa5e6f99 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name; + column_name = column_name.toUpperCase(); } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName); + getFullField().add(columnName.toUpperCase()); } } } From 339ea0265f2caab49ea5c37edb1e67a1702fcf4d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 19:01:05 +0800 Subject: [PATCH 33/78] remove toUppercase --- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..8e513adfc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); + column_name = column_name; } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); + getFullField().add(columnName); } } } From ead22fa6874a68aa69539156a7139d3e7366fe3d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 21:04:21 +0800 Subject: [PATCH 34/78] modify compare fun for repalce sql --- .../main/java/com/dtstack/flink/sql/Main.java | 2 +- .../sql/parser/CreateTmpTableParser.java | 2 +- .../flink/sql/parser/InsertSqlParser.java | 2 +- .../dtstack/flink/sql/side/SideSQLParser.java | 2 +- .../flink/sql/sink/oracle/OracleSink.java | 16 ++- .../sink/rdb/format/ExtendOutputFormat.java | 121 ++++++++++-------- .../sql/sink/sqlserver/SqlserverSink.java | 14 +- 7 files changed, 94 insertions(+), 65 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 2e9c979d1..19f660eb0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -188,7 +188,7 @@ public static void main(String[] args) throws Exception { String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index d205c59b6..5840b26a8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -69,7 +69,7 @@ public void parseSql(String sql, SqlTree sqlTree) { SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(selectSql,config); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 7d5c61e7a..52541385f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -60,7 +60,7 @@ public static InsertSqlParser newInstance(){ public void parseSql(String sql, SqlTree sqlTree) { SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 472146c11..388bb5497 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -58,7 +58,7 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws Queue queueInfo = Queues.newLinkedBlockingQueue(); SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 4252f025c..8d2399469 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -94,7 +95,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -107,10 +108,10 @@ 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 || keyCols.contains(col)) { + if (keyCols == null || keyCols.size() == 0 || containsIgnoreCase(keyCols,col)) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -157,6 +158,15 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 8e513adfc..7c14807d2 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -37,67 +38,75 @@ public class ExtendOutputFormat extends RetractJDBCOutputFormat { - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); + @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; - } + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (containsIgnoreCase(value, 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); + /** + * 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; - } - map.get(indexName).add(column_name); - } + 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; + } + 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); - } - } - } + 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); - } - } - } + /** + * 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); + } + } + } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } } 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 2bb53064e..e07d49945 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 @@ -92,7 +92,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -108,7 +108,7 @@ public String getUpdateSql(List column, List fullColumn, String if (keyCols == null || keyCols.size() == 0) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -154,6 +154,16 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } From ab515e73b1e298f3195b3c4c758021cc7345d181 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 12 Dec 2018 13:51:52 +0800 Subject: [PATCH 35/78] add tmp table to cache --- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 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 080d3d48b..a38719ebc 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 @@ -67,6 +67,8 @@ public class SideSqlExec { private SideSQLParser sideSQLParser = new SideSQLParser(); + private Map localTableCache = Maps.newHashMap(); + public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache) throws Exception { @@ -75,7 +77,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl throw new RuntimeException("need to set localSqlPluginPath"); } - Map localTableCache = Maps.newHashMap(tableCache); + localTableCache.putAll(tableCache); Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); Object pollObj = null; @@ -501,7 +503,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, throw new RuntimeException("need to set localSqlPluginPath"); } - Map localTableCache = Maps.newHashMap(tableCache); + localTableCache.putAll(tableCache); Queue exeQueue = sideSQLParser.getExeQueue(result.getExecSql(), sideTableMap.keySet()); Object pollObj = null; @@ -540,6 +542,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, throw new RuntimeException("Fields mismatch"); } } + localTableCache.put(result.getTableName(), table); } From 3ae08ec17b0f0dcfbfce43644221bca285a3bc59 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 17:42:53 +0800 Subject: [PATCH 36/78] case upper and lower --- .../main/java/com/dtstack/flink/sql/Main.java | 8 +++++++- .../flink/sql/parser/CreateTmpTableParser.java | 8 ++++++-- .../flink/sql/parser/InsertSqlParser.java | 7 ++++++- .../dtstack/flink/sql/side/SideSQLParser.java | 7 ++++++- .../flink/sql/sink/oracle/OracleSink.java | 16 +++++++++++++--- .../sql/sink/rdb/format/ExtendOutputFormat.java | 12 ++++++++++-- .../flink/sql/sink/sqlserver/SqlserverSink.java | 12 ++++++++++-- 7 files changed, 58 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 610abf21f..91e5b6e07 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql; @@ -34,6 +34,7 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -185,6 +186,11 @@ public static void main(String[] args) throws Exception { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); + + org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 8da89b2be..43c607849 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -65,8 +66,11 @@ public void parseSql(String sql, SqlTree sqlTree) { tableName = matcher.group(1).toUpperCase(); selectSql = "select " + matcher.group(2); } - - SqlParser sqlParser = SqlParser.create(selectSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(selectSql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e08540c92..52541385f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlJoin; @@ -57,7 +58,11 @@ public static InsertSqlParser newInstance(){ @Override public void parseSql(String sql, SqlTree sqlTree) { - SqlParser sqlParser = SqlParser.create(sql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 7f165ac30..a14458d0f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -56,7 +57,11 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); - SqlParser sqlParser = SqlParser.create(exeSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 4252f025c..2ca020c54 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -94,7 +95,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -107,10 +108,10 @@ 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 || keyCols.contains(col)) { + if (keyCols == null || keyCols.size() == 0 || containsIgnoreCase(keyCols,col)) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null ||containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -157,6 +158,15 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..f288e50c0 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -45,7 +46,7 @@ public boolean isReplaceInsertQuery() throws SQLException { if (!getRealIndexes().isEmpty()) { for (List value : getRealIndexes().values()) { for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { + if (containsIgnoreCase(value, fieldName)) { return true; } } @@ -99,5 +100,12 @@ public void fillFullColumns() throws SQLException { } } - + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } } 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 2bb53064e..904cb0895 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 @@ -92,7 +92,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -108,7 +108,7 @@ public String getUpdateSql(List column, List fullColumn, String if (keyCols == null || keyCols.size() == 0) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -154,6 +154,14 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } From 7406d68de0c190416902f96369cd3d2e3840944a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 19:07:21 +0800 Subject: [PATCH 37/78] case upper and lower 2 --- core/src/main/java/com/dtstack/flink/sql/Main.java | 4 ++-- .../java/com/dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../com/dtstack/flink/sql/parser/CreateTmpTableParser.java | 4 ++-- .../main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 - .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- .../main/java/com/dtstack/flink/sql/table/AbsTableParser.java | 4 ---- 6 files changed, 6 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 91e5b6e07..415b6e2a9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -191,7 +191,7 @@ public static void main(String[] args) throws Exception { .configBuilder() .setLex(Lex.MYSQL) .build(); - SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); @@ -252,7 +252,7 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); } classLoader.loadClass(funcInfo.getClassName()); - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName().toUpperCase(), + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index d10d6825e..b5e4a4aa4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -54,7 +54,7 @@ public boolean verify(String sql) { public void parseSql(String sql, SqlTree sqlTree) { Matcher matcher = PATTERN.matcher(sql); if(matcher.find()){ - String tableName = matcher.group(1).toUpperCase(); + String tableName = matcher.group(1); String fieldsInfoStr = matcher.group(2); String propsStr = matcher.group(3); Map props = parseProp(propsStr); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 43c607849..cb5620d0a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -63,7 +63,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String selectSql = null; if(matcher.find()) { - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); selectSql = "select " + matcher.group(2); } SqlParser.Config config = SqlParser @@ -93,7 +93,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String fieldsInfoStr = null; if (matcher.find()){ - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); fieldsInfoStr = matcher.group(2); } CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index a14458d0f..388bb5497 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -53,7 +53,6 @@ public class SideSQLParser { public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - exeSql = DtStringUtil.replaceIgnoreQuota(exeSql, "`", ""); System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); 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 a38719ebc..3623ecfd4 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 @@ -645,7 +645,7 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab String[] filedNameArr = new String[filed.length - 1]; System.arraycopy(filed, 0, filedNameArr, 0, filed.length - 1); String fieldName = String.join(" ", filedNameArr); - fieldNames.add(fieldName.toUpperCase()); + fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); Class tableField = table.getSchema().getType(i).get().getTypeClass(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 2fc2d799b..a43769918 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -82,10 +82,6 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); - if(fieldNameNeedsUpperCase()) { - fieldRow = fieldRow.toUpperCase(); - } - boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); if(isMatcherKey){ From 9a83ecc691ec2795d20fb698aee86e13b9d36ddd Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 20:07:34 +0800 Subject: [PATCH 38/78] fix conflict --- .../java/com/dtstack/flink/sql/parser/InsertSqlParser.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 61ee3f9eb..f62b09d04 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -21,12 +21,7 @@ package com.dtstack.flink.sql.parser; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; From df96ed2ae0a0adcdfd6569a9aa047ec5c1f37731 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:48:56 +0800 Subject: [PATCH 39/78] fix kafka offset reset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 1b60ec476..6c1aae76f 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From dc25cf552074bd46b2e156c6012556f71b717737 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:50:29 +0800 Subject: [PATCH 40/78] fix kafka offset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index de79e0509..768b30360 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From 110f3529ba5ceb4d263978637128961699118dc4 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:51:13 +0800 Subject: [PATCH 41/78] fix kafka offset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 131941467..e39a389e1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From a4b18db48ea87e14065ca8f72074c6afd53e6618 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 13 Dec 2018 15:16:59 +0800 Subject: [PATCH 42/78] support int --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 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 71607e918..962054f24 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 @@ -166,7 +166,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]); - jedis.set(key.toString(), (String) row.getField(i)); + jedis.set(key.toString(), row.getField(i).toString()); } outRecords.inc(); } From 0016a51db36d9971e7fdbe8b5c72bc239195020f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Dec 2018 16:40:22 +0800 Subject: [PATCH 43/78] Update README.md --- README.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5dfa80efb..93edde5c3 100644 --- a/README.md +++ b/README.md @@ -14,10 +14,13 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 - * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) + * serverSocket 源表 + * console 结果表 + * kafka avro格式 + * topN ## 1 快速起步 ### 1.1 运行模式 From 387cd28dd5a41117fae06ad7b18c79eeb5d828b9 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 17 Dec 2018 16:31:38 +0800 Subject: [PATCH 44/78] youhua --- .../flink/sql/launcher/LauncherMain.java | 56 ++++++++----------- .../flink/sql/launcher/LauncherOptions.java | 10 ++++ 2 files changed, 33 insertions(+), 33 deletions(-) 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 96eb6fb43..ec6a3e10c 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 @@ -62,50 +62,40 @@ 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(); List argList = optionParser.getProgramExeArgList(); - if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - 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); - pluginRoot = launcherOptions.getLocalSqlPluginPath(); - jarFile = new File(getLocalCoreJarPath(pluginRoot)); - remoteArgs = argList.toArray(new String[argList.size()]); - program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + }else{ + 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()))); } - //final JobGraph jobGraph; - //jobGraph = PackagedProgramUtils.createJobGraph(program, new Configuration(), 1); - //clusterClient.runDetached(jobGraph,null); - clusterClient.run(program, 1); - clusterClient.shutdown(); - - System.exit(0); + if(mode.equals(ClusterMode.yarnPer.name())){ + String flinkConfDir = launcherOptions.getFlinkconf(); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, launcherOptions.getDefaultParallelism()); + PerJobSubmitter.submit(launcherOptions, jobGraph); + } else { + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); + pluginRoot = launcherOptions.getLocalSqlPluginPath(); + jarFile = new File(getLocalCoreJarPath(pluginRoot)); + remoteArgs = argList.toArray(new String[argList.size()]); + 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, launcherOptions.getDefaultParallelism()); + clusterClient.shutdown(); + System.exit(0); + } } - System.out.println("---submit end----"); } 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 5cc762ebd..d9c5bd1ed 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 @@ -53,6 +53,8 @@ public class LauncherOptions { private String queue; + private int defaultParallelism=1; + public String getMode() { return mode; } @@ -156,4 +158,12 @@ public String getQueue() { public void setQueue(String queue) { this.queue = queue; } + + public int getDefaultParallelism() { + return defaultParallelism; + } + + public void setDefaultParallelism(int defaultParallelism) { + this.defaultParallelism = defaultParallelism; + } } From 91d089079b250321255fbe823960a4f130f40237 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 10:54:43 +0800 Subject: [PATCH 45/78] add kafka sink and read csv,txt format data --- .../flink/sql/sink/StreamSinkFactory.java | 7 +- .../dtstack/flink/sql/table/TableInfo.java | 50 ++++ .../dtstack/flink/sql/util/DtStringUtil.java | 27 +- docs/kafkaSource.md | 263 ++++++++++++++++-- kafka08/kafka08-sink/pom.xml | 94 +++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka08/kafka08-source/pom.xml | 91 ++++++ .../flink/sql/source/kafka/KafkaSource.java | 136 +++++++++ .../consumer/CustomerCommonConsumer.java | 68 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 74 +++++ .../kafka/consumer/CustomerJsonConsumer.java | 73 +++++ .../CustomerCommonDeserialization.java | 91 ++++++ .../CustomerCsvDeserialization.java | 119 ++++++++ .../CustomerJsonDeserialization.java | 141 ++++++++++ .../source/kafka/table/KafkaSourceParser.java | 58 ++++ .../kafka/table/KafkaSourceTableInfo.java | 83 ++++++ kafka08/pom.xml | 37 +++ kafka09/kafka09-sink/pom.xml | 90 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 61 ++++ kafka09/kafka09-source/pom.xml | 10 - .../flink/sql/source/kafka/KafkaSource.java | 72 +++-- .../consumer/CustomerCommonConsumer.java | 74 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 80 ++++++ .../CustomerJsonConsumer.java} | 21 +- .../CustomerCommonDeserialization.java | 150 ++++++++++ .../CustomerCsvDeserialization.java | 181 ++++++++++++ .../CustomerJsonDeserialization.java | 3 +- .../source/kafka/table/KafkaSourceParser.java | 24 +- .../kafka/table/KafkaSourceTableInfo.java | 97 ++----- kafka09/pom.xml | 8 + kafka10/kafka10-sink/pom.xml | 89 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka10/kafka10-source/pom.xml | 11 - .../flink/sql/source/kafka/KafkaSource.java | 75 +++-- .../consumer/CustomerCommonConsumer.java | 74 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 80 ++++++ .../kafka/consumer/CustomerJsonConsumer.java | 78 ++++++ .../CustomerCommonDeserialization.java | 150 ++++++++++ .../CustomerCsvDeserialization.java | 181 ++++++++++++ .../CustomerJsonDeserialization.java | 5 +- .../source/kafka/table/KafkaSourceParser.java | 24 +- .../kafka/table/KafkaSourceTableInfo.java | 97 ++----- kafka10/pom.xml | 7 + kafka11/kafka11-sink/pom.xml | 89 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 137 +++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 134 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka11/kafka11-source/pom.xml | 9 - .../flink/sql/source/kafka/KafkaSource.java | 69 +++-- .../consumer/CustomerCommonConsumer.java | 73 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 33 ++- .../CustomerJsonConsumer.java} | 13 +- .../CustomerCommonDeserialization.java | 152 ++++++++++ .../CustomerCsvDeserialization.java | 183 ++++++++++++ .../CustomerJsonDeserialization.java | 3 +- .../source/kafka/table/KafkaSourceParser.java | 45 +-- .../kafka/table/KafkaSourceTableInfo.java | 96 ++----- kafka11/pom.xml | 9 + pom.xml | 1 + 69 files changed, 4948 insertions(+), 389 deletions(-) create mode 100644 kafka08/kafka08-sink/pom.xml create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka08/kafka08-source/pom.xml create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java create mode 100644 kafka08/pom.xml create mode 100644 kafka09/kafka09-sink/pom.xml create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/{CustomerKafka09Consumer.java => consumer/CustomerJsonConsumer.java} (69%) create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename {kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka => kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization}/CustomerJsonDeserialization.java (98%) create mode 100644 kafka10/kafka10-sink/pom.xml create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename {kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka => kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization}/CustomerJsonDeserialization.java (98%) create mode 100644 kafka11/kafka11-sink/pom.xml create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java rename kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java => kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java (70%) rename kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/{CustomerKafka011Consumer.java => consumer/CustomerJsonConsumer.java} (86%) create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/{ => deserialization}/CustomerJsonDeserialization.java (98%) diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index 9ef30f97c..e072a2345 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -51,9 +51,9 @@ public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) DtClassLoader dtClassLoader = (DtClassLoader) classLoader; String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); + String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); Class targetParser = dtClassLoader.loadClass(className); if(!AbsTableParser.class.isAssignableFrom(targetParser)){ @@ -76,8 +76,9 @@ public static TableSink getTableSink(TargetTableInfo targetTableInfo, String loc String pluginJarDirPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), localSqlRootDir); PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); - String className = PluginUtil.getGenerClassName(pluginType, CURR_TYPE); + String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); Class sinkClass = dtClassLoader.loadClass(className); if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index b56b790ea..d69b5c2bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -36,6 +36,14 @@ public abstract class TableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; + public static final String SOURCE_DATA_TYPE = "sourcedatatype"; + + public static final String SINK_DATA_TYPE = "sinkdatatype"; + + public static final String FIELD_DELINITER = "fielddelimiter"; + + public static final String LENGTH_CHECK_POLICY = "lengthcheckpolicy"; + private String name; private String type; @@ -56,6 +64,16 @@ public abstract class TableInfo implements Serializable { private Integer parallelism = 1; + private String sourceDataType = "json"; + + private String sinkDataType = "json"; + + private String fieldDelimiter; + + private String lengthCheckPolicy = "SKIP"; + + + public String[] getFieldTypes() { return fieldTypes; } @@ -146,6 +164,38 @@ public List getFieldClassList() { return fieldClassList; } + public String getSourceDataType() { + return sourceDataType; + } + + public void setSourceDataType(String sourceDataType) { + this.sourceDataType = sourceDataType; + } + + public String getSinkDataType() { + return sinkDataType; + } + + public void setSinkDataType(String sinkDataType) { + this.sinkDataType = sinkDataType; + } + + public String getFieldDelimiter() { + return fieldDelimiter; + } + + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } + + public String getLengthCheckPolicy() { + return lengthCheckPolicy; + } + + public void setLengthCheckPolicy(String lengthCheckPolicy) { + this.lengthCheckPolicy = lengthCheckPolicy; + } + public void finish(){ this.fields = fieldList.toArray(new String[fieldList.size()]); this.fieldClasses = fieldClassList.toArray(new Class[fieldClassList.size()]); 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 959d4b13c..96026ca4b 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 @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.util; @@ -25,7 +25,7 @@ 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.sql.Timestamp; import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; @@ -225,4 +225,27 @@ public static boolean isJosn(String str){ } return flag; } + + public static Object parse(String str,Class clazz){ + String fieldType = clazz.getName(); + Object object = null; + if(fieldType.equals(Integer.class.getName())){ + object = Integer.parseInt(str); + }else if(fieldType.equals(Long.class.getName())){ + object = Long.parseLong(str); + }else if(fieldType.equals(Byte.class.getName())){ + object = str.getBytes()[0]; + }else if(fieldType.equals(String.class.getName())){ + object = str; + }else if(fieldType.equals(Float.class.getName())){ + object = Float.parseFloat(str); + }else if(fieldType.equals(Double.class.getName())){ + object = Double.parseDouble(str); + }else if (fieldType.equals(Timestamp.class.getName())){ + object = Timestamp.valueOf(str); + }else{ + throw new RuntimeException("no support field type for sql. the input type:" + fieldType); + } + return object; + } } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 748948a7b..d6745c3d7 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -1,3 +1,4 @@ +# 一、json格式数据源 ## 1.格式: ``` 数据现在支持json格式{"xx":"bb","cc":"dd"} @@ -9,16 +10,17 @@ CREATE TABLE tableName( WATERMARK FOR colName AS withOffset( colName , delayTime ) )WITH( type ='kafka09', - bootstrapServers ='ip:port,ip:port...', - zookeeperQuorum ='ip:port,ip:port/zkparent', - offsetReset ='latest', - topic ='topicName', - parallelism ='parllNum' + kafka.bootstrap.servers ='ip:port,ip:port...', + kafka.zookeeper.quorum ='ip:port,ip:port/zkparent', + kafka.auto.offset.reset ='latest', + kafka.topic ='topicName', + parallelism ='parllNum', + sourcedatatype ='json' #可不设置 ); ``` ## 2.支持的版本 - kafka09,kafka10,kafka11 + kafka08,kafka09,kafka10,kafka11 ## 3.表结构定义 @@ -35,14 +37,16 @@ CREATE TABLE tableName( |参数名称|含义|是否必填|默认值| |----|---|---|---| |type | kafka09 | 是|| -|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| -|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| -|topic | 需要读取的 topic 名称|是|| -|topicIsPattern | topic是否是正则表达式格式|否| false -|groupId | 需要读取的 groupId 名称|否|| -|offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| +|kafka.group.id | 需要读取的 groupId 名称|否|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|patterntopic | topic是否是正则表达式格式(true|false) |否| false +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| - +|sourcedatatype | 数据类型|否|json| +**kafka相关参数可以自定义,使用kafka.开头即可。** + ## 5.样例: ``` CREATE TABLE MyTable( @@ -53,13 +57,230 @@ CREATE TABLE MyTable( CHARACTER_LENGTH(channel) AS timeLeng )WITH( type ='kafka09', - bootstrapServers ='172.16.8.198:9092', - zookeeperQuorum ='172.16.8.198:2181/kafka', - offsetReset ='latest', - groupId='nbTest', - topic ='nbTest1,nbTest2,nbTest3', - --- topic ='mqTest.*', - ---topicIsPattern='true', - parallelism ='1' + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1,nbTest2,nbTest3', + --kafka.topic ='mqTest.*', + --patterntopic='true' + parallelism ='1', + sourcedatatype ='json' #可不设置 ); ``` +# 二、csv格式数据源 +根据字段分隔符进行数据分隔,按顺序匹配sql中配置的列。如数据分隔列数和sql中配置的列数相等直接匹配;如不同参照lengthcheckpolicy策略处理。 +## 1.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|parallelism | 并行度设置 |否|1| +|sourcedatatype | 数据类型|是 |csv| +|fielddelimiter | 字段分隔符|是 || +|lengthcheckpolicy | 单行字段条数检查策略 |否|可选,默认为SKIP,其它可选值为EXCEPTION、PAD。SKIP:字段数目不符合时跳过 。EXCEPTION:字段数目不符合时抛出异常。PAD:按顺序填充,不存在的置为null。| +**kafka相关参数可以自定义,使用kafka.开头即可。** + +## 2.样例: +``` +CREATE TABLE MyTable( + name varchar, + channel varchar, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1', + --kafka.topic ='mqTest.*', + --kafka.topicIsPattern='true' + parallelism ='1', + sourcedatatype ='csv', + fielddelimiter ='\|', + lengthcheckpolicy = 'PAD' + ); + ``` +# 三、text格式数据源UDF自定义拆分 +Kafka源表数据解析流程:Kafka Source Table -> UDTF ->Realtime Compute -> SINK。从Kakfa读入的数据,都是VARBINARY(二进制)格式,对读入的每条数据,都需要用UDTF将其解析成格式化数据。 + 与其他格式不同,本格式定义DDL必须与以下SQL一摸一样,表中的五个字段顺序务必保持一致: + +## 1. 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 +``` +create table kafka_stream( + _topic STRING, + _messageKey STRING, + _message STRING, + _partition INT, + _offset BIGINT, +) with ( + type ='kafka09', + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1', + parallelism ='1', + sourcedatatype='text' + ) +``` +## 2.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|parallelism | 并行度设置|否|1| +|sourcedatatype | 数据类型|否|text| +**kafka相关参数可以自定义,使用kafka.开头即可。** + +## 2.自定义: +从kafka读出的数据,需要进行窗口计算。 按照实时计算目前的设计,滚窗/滑窗等窗口操作,需要(且必须)在源表DDL上定义Watermark。Kafka源表比较特殊。如果要以kafka中message字段中的的Event Time进行窗口操作, +需要先从message字段,使用UDX解析出event time,才能定义watermark。 在kafka源表场景中,需要使用计算列。 假设,kafka中写入的数据如下: +2018-11-11 00:00:00|1|Anna|female整个计算流程为:Kafka SOURCE->UDTF->Realtime Compute->RDS SINK(单一分隔符可直接使用类csv格式模板,自定义适用于更复杂的数据类型,本说明只做参考) + +**SQL** +``` +-- 定义解析Kakfa message的UDTF + CREATE FUNCTION kafkapaser AS 'com.XXXX.kafkaUDTF'; + CREATE FUNCTION kafkaUDF AS 'com.XXXX.kafkaUDF'; + -- 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 + create table kafka_src ( + _topic STRING, + _messageKey STRING, + _message STRING, + _partition INT, + _offset BIGINT, + ctime AS TO_TIMESTAMP(kafkaUDF(_message)), -- 定义计算列,计算列可理解为占位符,源表中并没有这一列,其中的数据可经过下游计算得出。注意计算里的类型必须为timestamp才能在做watermark。 + watermark for ctime as withoffset(ctime,0) -- 在计算列上定义watermark + ) WITH ( + type = 'kafka010', -- Kafka Source类型,与Kafka版本强相关,目前支持的Kafka版本请参考本文档 + topic = 'test_kafka_topic', + ... + ); + create table rds_sink ( + name VARCHAR, + age INT, + grade VARCHAR, + updateTime TIMESTAMP + ) WITH( + type='mysql', + url='jdbc:mysql://localhost:3306/test', + tableName='test4', + userName='test', + password='XXXXXX' + ); + -- 使用UDTF,将二进制数据解析成格式化数据 + CREATE VIEW input_view ( + name, + age, + grade, + updateTime + ) AS + SELECT + COUNT(*) as cnt, + T.ctime, + T.order, + T.name, + T.sex + from + kafka_src as S, + LATERAL TABLE (kafkapaser _message)) as T ( + ctime, + order, + name, + sex + ) + Group BY T.sex, + TUMBLE(ctime, INTERVAL '1' MINUTE); + -- 对input_view中输出的数据做计算 + CREATE VIEW view2 ( + cnt, + sex + ) AS + SELECT + COUNT(*) as cnt, + T.sex + from + input_view + Group BY sex, TUMBLE(ctime, INTERVAL '1' MINUTE); + -- 使用解析出的格式化数据进行计算,并将结果输出到RDS中 + insert into rds_sink + SELECT + cnt,sex + from view2; + ``` +**UDF&UDTF** +``` +package com.XXXX; + import com.XXXX.fastjson.JSONObject; + import org.apache.flink.table.functions.TableFunction; + import org.apache.flink.table.types.DataType; + import org.apache.flink.table.types.DataTypes; + import org.apache.flink.types.Row; + import java.io.UnsupportedEncodingException; + /** + 以下例子解析输入Kafka中的JSON字符串,并将其格式化输出 + **/ + public class kafkaUDTF extends TableFunction { + public void eval(byte[] message) { + try { + // 读入一个二进制数据,并将其转换为String格式 + String msg = new String(message, "UTF-8"); + // 提取JSON Object中各字段 + String ctime = Timestamp.valueOf(data.split('\\|')[0]); + String order = data.split('\\|')[1]; + String name = data.split('\\|')[2]; + String sex = data.split('\\|')[3]; + // 将解析出的字段放到要输出的Row()对象 + Row row = new Row(4); + row.setField(0, ctime); + row.setField(1, age); + row.setField(2, grade); + row.setField(3, updateTime); + System.out.println("Kafka message str ==>" + row.toString()); + // 输出一行 + collect(row); + } catch (ClassCastException e) { + System.out.println("Input data format error. Input data " + msg + "is not json string"); + } + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + } + @Override + // 如果返回值是Row,就必须重载实现这个方法,显式地告诉系统返回的字段类型 + // 定义输出Row()对象的字段类型 + public DataType getResultType(Object[] arguments, Class[] argTypes) { + return DataTypes.createRowType(DataTypes.TIMESTAMP,DataTypes.STRING, DataTypes.Integer, DataTypes.STRING,DataTypes.STRING); + } + } + + package com.dp58; + package com.dp58.sql.udx; + import org.apache.flink.table.functions.FunctionContext; + import org.apache.flink.table.functions.ScalarFunction; + public class KafkaUDF extends ScalarFunction { + // 可选,open方法可以不写 + // 需要import org.apache.flink.table.functions.FunctionContext; + public String eval(byte[] message) { + // 读入一个二进制数据,并将其转换为String格式 + String msg = new String(message, "UTF-8"); + return msg.split('\\|')[0]; + } + public long eval(String b, String c) { + return eval(b) + eval(c); + } + //可选,close方法可以不写 + @Override + public void close() { + } + } + ``` diff --git a/kafka08/kafka08-sink/pom.xml b/kafka08/kafka08-sink/pom.xml new file mode 100644 index 000000000..47391d182 --- /dev/null +++ b/kafka08/kafka08-sink/pom.xml @@ -0,0 +1,94 @@ + + + + sql.kafka08 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka08 + jar + + kafka08-sink + http://maven.apache.org + + + + org.apache.flink + flink-connector-kafka-0.8_2.11 + ${flink.version} + + + org.apache.flink + flink-json + ${flink.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/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..41f597fe1 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka08TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka08SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka08SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka08SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka08SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka08SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka08SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka08SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka08SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka08TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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.kafka.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; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..67100dc05 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,62 @@ +/* + * 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.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + // Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka08/kafka08-source/pom.xml b/kafka08/kafka08-source/pom.xml new file mode 100644 index 000000000..df98c2603 --- /dev/null +++ b/kafka08/kafka08-source/pom.xml @@ -0,0 +1,91 @@ + + + + sql.kafka08 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.source.kafka08 + jar + + kafka08-source + http://maven.apache.org + + + + org.apache.flink + flink-connector-kafka-0.8_2.11 + 1.6.0 + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + 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/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java new file mode 100644 index 000000000..4ec6832a2 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -0,0 +1,136 @@ +/* + * 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.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +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; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +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; +import java.util.regex.Pattern; + +public class KafkaSource implements IStreamSourceGener
{ + + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafka08SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka08SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka08SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka08SourceTableInfo.getPatternTopic(); + + Properties props = new Properties(); + for (String key : kafka08SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka08SourceTableInfo.getKafkaParam(key)); + } + + TypeInformation[] types = new TypeInformation[kafka08SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka08SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka08SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka08SourceTableInfo.getFields()); + + FlinkKafkaConsumer08 kafkaSrc; + String fields = StringUtils.join(kafka08SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization(typeInformation, + kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); + } + } else { + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + + } + + //earliest,latest + if ("earliest".equalsIgnoreCase(offsetReset)) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} + try { + Properties properties = PluginUtil.jsonStrToObject(offsetReset, 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:" + offsetReset); + } + } else { + kafkaSrc.setStartFromLatest(); + } + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka08SourceTableInfo.getParallelism(); + if (parallelism != null) { + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..b1e6f24c7 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,68 @@ +/** + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer08 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..7457dd9ff --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -0,0 +1,74 @@ +/* + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java new file mode 100644 index 000000000..0b775a4dc --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -0,0 +1,73 @@ +/* + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerJsonConsumer extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } + + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..0d08dd092 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,91 @@ +/* + * 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.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..caa0ce7c6 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,119 @@ +/* + * 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.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private static final long serialVersionUID = 2385115520960444192L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + public CustomerJsonDeserialization(TypeInformation typeInfo){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + + JsonNode root = objectMapper.readTree(message); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + + if (node == null) { + if (failOnMissingField) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + + numInResolveRecord.inc(); + return row; + } catch (Throwable t) { + //add metric of dirty data + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + + return null; + + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java new file mode 100644 index 000000000..668cbcfd9 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.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.source.kafka.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + + +public class KafkaSourceParser extends AbsSourceParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka08SourceTableInfo = new KafkaSourceTableInfo(); + kafka08SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka08SourceTableInfo); + kafka08SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + + kafka08SourceTableInfo.setPatternTopic(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.PATTERNTOPIC_KEY.toLowerCase()))); + + if (props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE) != null) { + kafka08SourceTableInfo.setSourceDataType(props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE).toString()); + } + if (props.get(KafkaSourceTableInfo.FIELD_DELINITER) != null) { + kafka08SourceTableInfo.setFieldDelimiter(props.get(KafkaSourceTableInfo.FIELD_DELINITER).toString()); + } + if (props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY) != null) { + kafka08SourceTableInfo.setLengthCheckPolicy(props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY).toString()); + } + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka08SourceTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka08SourceTableInfo; + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java new file mode 100644 index 000000000..4fb59f7d2 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -0,0 +1,83 @@ +/* + * 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.source.kafka.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + + +public class KafkaSourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka08"; + + public static final String PATTERNTOPIC_KEY = "patterntopic"; + + private Boolean patternTopic=false; + + public Boolean getPatternTopic() { + return patternTopic; + } + + public void setPatternTopic(Boolean patternTopic) { + if (patternTopic==null){ + return; + } + this.patternTopic = patternTopic; + } + + public KafkaSourceTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka08/pom.xml b/kafka08/pom.xml new file mode 100644 index 000000000..ced039c1c --- /dev/null +++ b/kafka08/pom.xml @@ -0,0 +1,37 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.kafka08 + pom + + + kafka08-source + kafka08-sink + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml new file mode 100644 index 000000000..5d11d9df8 --- /dev/null +++ b/kafka09/kafka09-sink/pom.xml @@ -0,0 +1,90 @@ + + + + sql.kafka09 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka09 + jar + + kafka09-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.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/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..2500bad39 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka09SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka09SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka09SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka09TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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.kafka.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; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..6643204a9 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,61 @@ +/* + * 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.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index 59e05c020..6f26b62a0 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -14,16 +14,6 @@ kafka09-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.9_2.11 - ${flink.version} - - - - 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 b08485e0b..c37da6791 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 @@ -21,15 +21,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; 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.BooleanUtils; 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.datastream.DataStreamSource; 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; @@ -64,16 +69,15 @@ public class KafkaSource implements IStreamSourceGener

{ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafka09SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka09SourceTableInfo.getTopic(); + String topicName = kafka09SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka09SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka09SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka09SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + + for (String key:kafka09SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka09SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka09SourceTableInfo.getFields().length]; for(int i = 0; i< kafka09SourceTableInfo.getFieldClasses().length; i++){ @@ -81,21 +85,44 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); + + FlinkKafkaConsumer09 kafkaSrc; - if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + String fields=StringUtils.join(kafka09SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ + if("earliest".equalsIgnoreCase(offsetReset)){ kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka09SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} + }else if(DtStringUtil.isJosn(offsetReset)){// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for(Map.Entry entry:offsetMap.entrySet()){ @@ -103,14 +130,17 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka09SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } }else { kafkaSrc.setStartFromLatest(); } - - String fields = StringUtils.join(kafka09SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka09SourceTableInfo.getParallelism(); + if(parallelism != null){ + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..46272b72c --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,74 @@ +/** + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer09 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..28b374c43 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer09 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCsvDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java similarity index 69% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 09e684c75..55b1ba851 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -16,12 +16,14 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; @@ -37,35 +39,38 @@ /** * Reason: - * Date: 2018/10/12 + * Date: 2018/10/19 * Company: www.dtstack.com + * * @author xuchao */ -public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { +public class CustomerJsonConsumer extends FlinkKafkaConsumer09 { - private static final long serialVersionUID = 4451177393982291909L; + private static final long serialVersionUID = -2265366268827807739L; private CustomerJsonDeserialization customerJsonDeserialization; - public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } - public CustomerKafka09Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + @Override - public void run(SourceContext sourceContext) throws Exception { + public void run(SourceFunction.SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); customerJsonDeserialization.initMetric(); super.run(sourceContext); } @Override - protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); customerJsonDeserialization.setFetcher(fetcher); return fetcher; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..cc23e69cf --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,150 @@ +/* + * 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.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..38b874ed9 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,181 @@ +/* + * 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.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index bcb54159a..24b607ee1 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -123,6 +123,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 8800dc18e..0b7edcdaf 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,12 +41,24 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } - public String getTopic() { - return topic; + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); } - public void setTopic(String topic) { - this.topic = topic; + public String getKafkaParam(String key){ + return kafkaParam.get(key); } - public String getGroupId() { - return groupId; + public Boolean getKafkaBooleanParam(String key){ + return Boolean.valueOf(kafkaParam.getOrDefault(key,"false").toLowerCase()); } - - public void setGroupId(String groupId) { - this.groupId = groupId; - } - - public String getOffsetReset() { - return offsetReset; - } - - public void setOffsetReset(String offsetReset) { - if (offsetReset == null) { - return; - } - - this.offsetReset = offsetReset; - } - - public String getOffset() { - return offset; - } - - public void setOffset(String offset) { - this.offset = offset; + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 457a8d387..1ae761e9b 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -15,9 +15,17 @@ kafka09-source + kafka09-sink + + + org.apache.flink + flink-connector-kafka-0.9_2.11 + ${flink.version} + + junit junit diff --git a/kafka10/kafka10-sink/pom.xml b/kafka10/kafka10-sink/pom.xml new file mode 100644 index 000000000..3a8b7592e --- /dev/null +++ b/kafka10/kafka10-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka10 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka10 + jar + + kafka10-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.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/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..5f0927960 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka010SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka010SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka010SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka010SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka010SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka010SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka010SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka010SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka010TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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.kafka.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; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..15a29cde3 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,62 @@ +/* + * 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.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index 365f98048..0f0c55c36 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -13,17 +13,6 @@ kafka10-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.10_2.11 - ${flink.version} - - - - - 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 523eb25dc..36c1e5e6e 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 @@ -20,14 +20,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; 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.BooleanUtils; 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.datastream.DataStreamSource; 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; @@ -63,40 +69,63 @@ public class KafkaSource implements IStreamSourceGener

{ @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka010SourceTableInfo.getTopic(); + String topicName = kafka010SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka010SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka010SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka010SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + for (String key:kafka010SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka010SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; - for (int i = 0; i < kafka010SourceTableInfo.getFieldClasses().length; i++) { + + for(int i = 0; i< kafka010SourceTableInfo.getFieldClasses().length; i++){ types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); } TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); FlinkKafkaConsumer010 kafkaSrc; - if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + + String fields= StringUtils.join(kafka010SourceTableInfo.getFields(), ","); + + + if ("json".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if ("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())) { + if("earliest".equalsIgnoreCase(offsetReset)){ kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for (Map.Entry entry : offsetMap.entrySet()) { @@ -104,14 +133,18 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } - } else { + }else{ kafkaSrc.setStartFromLatest(); } - String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka010SourceTableInfo.getParallelism(); + if(parallelism != null){ + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..6faf68b1b --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,74 @@ +/** + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer010 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..cc5916849 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer010 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCsvDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java new file mode 100644 index 000000000..e41337885 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -0,0 +1,78 @@ +/* + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerJsonConsumer extends FlinkKafkaConsumer010 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..cc23e69cf --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,150 @@ +/* + * 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.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..38b874ed9 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,181 @@ +/* + * 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.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index ec369b28f..24b607ee1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -16,9 +16,9 @@ * limitations under the License. */ + - -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -123,6 +123,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ed89d8ce4..acc2f795d 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,12 +41,24 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); - public void setGroupId(String groupId) { - this.groupId = groupId; - } - public String getOffsetReset() { - return offsetReset; + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); } - public void setOffsetReset(String offsetReset) { - if(offsetReset == null){ - return; - } - this.offsetReset = offsetReset; + public String getKafkaParam(String key){ + return kafkaParam.get(key); } - public String getOffset() { - return offset; + public Boolean getKafkaBooleanParam(String key){ + return Boolean.valueOf(kafkaParam.getOrDefault(key,"false").toLowerCase()); } - public void setOffset(String offset) { - if (offsetReset == null) { - return; - } - this.offset = offset; + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); } - public Boolean getTopicIsPattern() { - return topicIsPattern; - } - public void setTopicIsPattern(Boolean topicIsPattern) { - this.topicIsPattern = topicIsPattern; - } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka10/pom.xml b/kafka10/pom.xml index 9c8c78bba..2f85f5c3b 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -14,9 +14,16 @@ kafka10-source + kafka10-sink + + org.apache.flink + flink-connector-kafka-0.10_2.11 + ${flink.version} + + junit junit diff --git a/kafka11/kafka11-sink/pom.xml b/kafka11/kafka11-sink/pom.xml new file mode 100644 index 000000000..4c748c90b --- /dev/null +++ b/kafka11/kafka11-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka11 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka11 + 1.0-SNAPSHOT + kafka11-sink + jar + + + + + org.apache.flink + flink-json + ${flink.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/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..4f3715923 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,137 @@ +/* + * 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.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..cc13ac79e --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,134 @@ +/* + * 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.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; + +import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka011SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka011SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka011SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka011SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka011SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka011SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka011SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka011SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka011TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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.kafka.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; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..15a29cde3 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,62 @@ +/* + * 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.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index 98182aa64..ae6997f7e 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -13,15 +13,6 @@ kafka11-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.11_2.11 - ${flink.version} - - - 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 5c9f5eb49..f1ac74b01 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 @@ -20,14 +20,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; 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.BooleanUtils; 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.datastream.DataStreamSource; 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; @@ -64,16 +70,14 @@ public class KafkaSource implements IStreamSourceGener

{ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka011SourceTableInfo.getTopic(); + String topicName = kafka011SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka011SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka011SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka011SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + for (String key : kafka011SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka011SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; for (int i = 0; i < kafka011SourceTableInfo.getFieldClasses().length; i++) { @@ -83,21 +87,41 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); FlinkKafkaConsumer011 kafkaSrc; - if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); - } + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if ("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())) { + if ("earliest".equalsIgnoreCase(offsetReset)) { kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for (Map.Entry entry : offsetMap.entrySet()) { @@ -105,14 +129,17 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } } else { kafkaSrc.setStartFromLatest(); } - - String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka011SourceTableInfo.getParallelism(); + if (parallelism != null) { + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..58191142f --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,73 @@ +/** + * 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.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer011 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java similarity index 70% rename from kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index f6878473b..e704d1f26 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -16,17 +16,20 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; @@ -39,36 +42,40 @@ * Reason: * Date: 2018/10/19 * Company: www.dtstack.com + * * @author xuchao */ -public class CustomerKafka010Consumer extends FlinkKafkaConsumer010 { +public class CustomerCsvConsumer extends FlinkKafkaConsumer011 { - private static final long serialVersionUID = 4873757508981691375L; + private static final long serialVersionUID = -2265366268827807739L; - private CustomerJsonDeserialization customerJsonDeserialization; + private CustomerCsvDeserialization customerCsvDeserialization; - public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); - this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; } - public CustomerKafka010Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); - this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; } + + + + @Override public void run(SourceContext sourceContext) throws Exception { - customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); - customerJsonDeserialization.initMetric(); + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); super.run(sourceContext); } @Override protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); - customerJsonDeserialization.setFetcher(fetcher); + customerCsvDeserialization.setFetcher(fetcher); return fetcher; } - } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java similarity index 86% rename from kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 7cc1f6f40..568ef54c0 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -16,24 +16,24 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Collections; import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; @@ -46,22 +46,23 @@ * @author xuchao */ -public class CustomerKafka011Consumer extends FlinkKafkaConsumer011 { +public class CustomerJsonConsumer extends FlinkKafkaConsumer011 { private static final long serialVersionUID = -2265366268827807739L; private CustomerJsonDeserialization customerJsonDeserialization; - public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } - public CustomerKafka011Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + @Override public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..e15d3892a --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,152 @@ +/* + * 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.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..cf15b9904 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,183 @@ +/* + * 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.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 78f1c17ff..e72fd0303 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -125,6 +125,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 47d453adc..f5431abbc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka.table; @@ -30,25 +29,37 @@ * Reason: * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSourceParser extends AbsSourceParser { - @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - - KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); - kafka11SourceTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); - - kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka11SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); - kafka11SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); - kafka11SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); - kafka11SourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); - kafka11SourceTableInfo.setTopicIsPattern(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.TOPICISPATTERN_KEY.toLowerCase()))); - kafka11SourceTableInfo.check(); - return kafka11SourceTableInfo; - } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); + kafka11SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); + kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + + kafka11SourceTableInfo.setPatternTopic(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.PATTERNTOPIC_KEY.toLowerCase()))); + + if (props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE) != null) { + kafka11SourceTableInfo.setSourceDataType(props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE).toString()); + } + if (props.get(KafkaSourceTableInfo.FIELD_DELINITER) != null) { + kafka11SourceTableInfo.setFieldDelimiter(props.get(KafkaSourceTableInfo.FIELD_DELINITER).toString()); + } + if (props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY) != null) { + kafka11SourceTableInfo.setLengthCheckPolicy(props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY).toString()); + } + for (String key : props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SourceTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SourceTableInfo; + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 1b60ec476..ffc270486 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -21,7 +21,9 @@ import com.dtstack.flink.sql.table.SourceTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; -import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; /** * Reason: @@ -32,105 +34,53 @@ */ public class KafkaSourceTableInfo extends SourceTableInfo { - //version private static final String CURR_TYPE = "kafka11"; - public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; - - public static final String TOPIC_KEY = "topic"; - - public static final String GROUPID_KEY = "groupId"; - - public static final String OFFSETRESET_KEY = "offsetReset"; + public static final String PATTERNTOPIC_KEY = "patterntopic"; - public static final String TOPICISPATTERN_KEY = "topicIsPattern"; + private Boolean patternTopic = false; - private String topic; - - private String groupId; - - private Boolean topicIsPattern = false; - - private String bootstrapServers; - public Boolean getTopicIsPattern() { - return topicIsPattern; + public Boolean getPatternTopic() { + return patternTopic; } - public void setTopicIsPattern(Boolean topicIsPattern) { - if (topicIsPattern == null) return; - - this.topicIsPattern = topicIsPattern; + public void setPatternTopic(Boolean patternTopic) { + if (patternTopic == null) { + return; + } + this.patternTopic = patternTopic; } - //latest, earliest - private String offsetReset = "latest"; - - private String offset; - public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } + public static Map kafkaParam = new HashMap(); - public String getBootstrapServers() { - return bootstrapServers; - } - - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - - public String getGroupId() { - return groupId; + public void addKafkaParam(String key, String value) { + kafkaParam.put(key, value); } - public void setGroupId(String groupId) { - this.groupId = groupId; + public String getKafkaParam(String key) { + return kafkaParam.get(key); } - public String getOffsetReset() { - return offsetReset; - } - - public void setOffsetReset(String offsetReset) { - if (offsetReset == null) { - return; - } - - this.offsetReset = offsetReset; - } - - public String getOffset() { - return offset; - } - - public void setOffset(String offset) { - this.offset = offset; + public Set getKafkaParamKeys() { + return kafkaParam.keySet(); } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka11/pom.xml b/kafka11/pom.xml index 841e20be9..f1f57aa03 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -14,9 +14,17 @@ kafka11-source + kafka11-sink + + org.apache.flink + flink-connector-kafka-0.11_2.11 + ${flink.version} + + + junit junit @@ -33,4 +41,5 @@ + \ No newline at end of file diff --git a/pom.xml b/pom.xml index df1a54646..58cf34f03 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,7 @@ sqlserver oracle cassandra + kafka08 From f925d41e181f786cb9075b6246ba0c22c02d4789 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 12:04:56 +0800 Subject: [PATCH 46/78] add class description --- .../dtstack/flink/sql/side/SideSqlExec.java | 1 + .../flink/sql/table/SourceTableInfo.java | 12 +++++++++ .../dtstack/flink/sql/table/TableInfo.java | 24 ----------------- .../flink/sql/table/TargetTableInfo.java | 12 +++++++++ docs/kafkaSource.md | 3 ++- .../sink/kafka/CustomerCsvSerialization.java | 8 ++++++ .../flink/sql/sink/kafka/KafkaSink.java | 12 ++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 26 +++++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 11 +++++++- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 16 +++--------- .../kafka/consumer/CustomerCsvConsumer.java | 6 ++--- .../CustomerCommonDeserialization.java | 6 +++-- .../CustomerCsvDeserialization.java | 5 ++-- .../source/kafka/table/KafkaSourceParser.java | 9 ++++++- .../sink/kafka/CustomerCsvSerialization.java | 9 ++++++- .../flink/sql/sink/kafka/KafkaSink.java | 13 +++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 24 +++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 11 ++++++-- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 7 ++--- .../kafka/consumer/CustomerCsvConsumer.java | 9 +++---- .../kafka/consumer/CustomerJsonConsumer.java | 7 ++--- .../CustomerCommonDeserialization.java | 7 +++-- .../CustomerCsvDeserialization.java | 7 ++--- .../source/kafka/table/KafkaSourceParser.java | 7 +++-- .../sink/kafka/CustomerCsvSerialization.java | 11 +++++++- .../flink/sql/sink/kafka/KafkaSink.java | 15 ++++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 24 +++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 13 ++++++++-- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 9 ++++--- .../kafka/consumer/CustomerCsvConsumer.java | 9 ++++--- .../CustomerCommonDeserialization.java | 7 ++++- .../CustomerCsvDeserialization.java | 8 ++++-- .../sink/kafka/CustomerCsvSerialization.java | 10 ++++++- .../flink/sql/sink/kafka/KafkaSink.java | 14 ++++++++-- .../sql/sink/kafka/table/KafkaSinkParser.java | 12 +++++++-- .../sink/kafka/table/KafkaSinkTableInfo.java | 10 ++++++- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 8 +++--- .../kafka/consumer/CustomerCsvConsumer.java | 8 +++--- .../CustomerCommonDeserialization.java | 6 ++++- .../CustomerCsvDeserialization.java | 8 +++--- 44 files changed, 296 insertions(+), 132 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 3623ecfd4..5e5534438 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 @@ -391,6 +391,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable getFieldClassList() { return fieldClassList; } - public String getSourceDataType() { - return sourceDataType; - } - - public void setSourceDataType(String sourceDataType) { - this.sourceDataType = sourceDataType; - } - public String getSinkDataType() { - return sinkDataType; - } - - public void setSinkDataType(String sinkDataType) { - this.sinkDataType = sinkDataType; - } public String getFieldDelimiter() { return fieldDelimiter; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java index a9a846707..70b625b5c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java @@ -30,4 +30,16 @@ public abstract class TargetTableInfo extends TableInfo { public static final String TARGET_SUFFIX = "Sink"; + + public static final String SINK_DATA_TYPE = "sinkdatatype"; + + private String sinkDataType = "json"; + + public String getSinkDataType() { + return sinkDataType; + } + + public void setSinkDataType(String sinkDataType) { + this.sinkDataType = sinkDataType; + } } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index d6745c3d7..2f6ce202c 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -20,7 +20,8 @@ CREATE TABLE tableName( ``` ## 2.支持的版本 - kafka08,kafka09,kafka10,kafka11 + kafka08,kafka09,kafka10,kafka11 + **kafka读取和写入的版本必须一致,否则会有兼容性错误。** ## 3.表结构定义 diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..53febb240 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -15,6 +15,14 @@ import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 41f597fe1..4aa7f49fa 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,13 @@ import java.util.Optional; import java.util.Properties; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +92,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka08SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka08SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka08SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka08SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka08SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..2b6c50512 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,35 @@ import java.util.Map; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka08SinkTableInfo = new KafkaSinkTableInfo(); + kafka08SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka08SinkTableInfo); + kafka08SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka08SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka08SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka08SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka08SinkTableInfo; } } diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 67100dc05..20e228d30 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -25,9 +25,18 @@ import java.util.Map; import java.util.Set; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka08"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 4ec6832a2..2e7136666 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -104,7 +104,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index b1e6f24c7..34b349e2c 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -18,29 +18,21 @@ package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi * - * @author maqi + * @modifyer maqi */ public class CustomerCommonConsumer extends FlinkKafkaConsumer08 { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index 7457dd9ff..ad859e485 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,11 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi * - * @author xuchao + * @modifyer maqi */ public class CustomerCsvConsumer extends FlinkKafkaConsumer08 { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index 0d08dd092..f35bacce7 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -32,9 +32,11 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 - * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index caa0ce7c6..e67d79400 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -33,10 +33,11 @@ import java.io.IOException; /** - * json string parsing custom - * Date: 2017/5/28 + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi */ public class CustomerCsvDeserialization extends AbsDeserialization { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 668cbcfd9..9af3c88ca 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -25,7 +25,14 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author xuchao + * + * @modifyer DocLi + */ public class KafkaSourceParser extends AbsSourceParser { diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..88a10a293 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -14,7 +14,14 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 2500bad39..106c3ab5c 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,14 @@ import java.util.Optional; import java.util.Properties; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +93,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka09SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka09SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka09SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka09SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka09SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..b51b9ea01 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,33 @@ import java.util.Map; +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka09SinkTableInfo = new KafkaSinkTableInfo(); + kafka09SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka09SinkTableInfo); + kafka09SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka09SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka09SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka09SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka09SinkTableInfo; } } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 6643204a9..bad84231d 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -24,10 +24,17 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka09"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); 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 c37da6791..c01366e4c 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 @@ -114,7 +114,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index 46272b72c..f6e0eb670 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -36,11 +36,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author maqi */ public class CustomerCommonConsumer extends FlinkKafkaConsumer09 { diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index 28b374c43..70fd50b7b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author xuchao */ public class CustomerCsvConsumer extends FlinkKafkaConsumer09 { @@ -62,8 +63,6 @@ public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization } - - @Override public void run(SourceFunction.SourceContext sourceContext) throws Exception { customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 55b1ba851..e3e9e2d8a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -38,11 +38,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com - * * @author xuchao + * + * @modifyer maqi + * */ public class CustomerJsonConsumer extends FlinkKafkaConsumer09 { diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index cc23e69cf..4a235e296 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -43,9 +43,12 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 - * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index 38b874ed9..20829bf61 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -44,12 +44,13 @@ import static com.dtstack.flink.sql.metric.MetricConstant.*; /** - * json string parsing custom - * Date: 2017/5/28 + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ - public class CustomerCsvDeserialization extends AbsDeserialization { private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 0b7edcdaf..3b0d3cc9a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -27,10 +27,13 @@ import java.util.Map; /** - * Reason: - * Date: 2018/7/4 + /** + * Date: 2018/12/18 * Company: www.dtstack.com * @author xuchao + * + * @modifyer Docli + * */ public class KafkaSourceParser extends AbsSourceParser { diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..825703332 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -14,7 +14,16 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 5f0927960..7d3748441 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,16 @@ import java.util.Optional; import java.util.Properties; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +95,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka010SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka010SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka010SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka010SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka010SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..c0818f34c 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,33 @@ import java.util.Map; +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka10SinkTableInfo = new KafkaSinkTableInfo(); + kafka10SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka10SinkTableInfo); + kafka10SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka10SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka10SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka10SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka10SinkTableInfo; } } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 15a29cde3..6557355d9 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -24,10 +24,19 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka10"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); 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 36c1e5e6e..586f046af 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 @@ -117,7 +117,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index 6faf68b1b..bbf91997a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -36,11 +36,14 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * + * Date: 2018/12/18 * Company: www.dtstack.com * - * @author maqi + * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonConsumer extends FlinkKafkaConsumer010 { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index cc5916849..c576a445c 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,14 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * + * Date: 2018/12/18 * Company: www.dtstack.com * - * @author xuchao + * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCsvConsumer extends FlinkKafkaConsumer010 { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index cc23e69cf..3502c959c 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -43,9 +43,14 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 + * + * Date: 2018/12/18 + * Company: www.dtstack.com * * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index 38b874ed9..50a8c5e12 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -44,10 +44,14 @@ import static com.dtstack.flink.sql.metric.MetricConstant.*; /** - * json string parsing custom - * Date: 2017/5/28 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCsvDeserialization extends AbsDeserialization { diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 4f3715923..5184eba4f 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -31,7 +31,15 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index cc13ac79e..44383ec9b 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -41,6 +41,16 @@ import java.util.Optional; import java.util.Properties; +/** + * kafka result table + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -87,9 +97,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka011SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka011SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka011SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..9bf83dccc 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -23,7 +23,15 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { @@ -32,7 +40,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index e704d1f26..66a83bdd1 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -39,11 +39,13 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author xuchao */ public class CustomerCsvConsumer extends FlinkKafkaConsumer011 { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index e15d3892a..7fc891638 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -45,9 +45,13 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index cf15b9904..cd0eb6be7 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -46,12 +46,14 @@ import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; /** - * json string parsing custom - * Date: 2017/5/28 + * + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ - public class CustomerCsvDeserialization extends AbsDeserialization { private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); From 774eb8415c6d90a58db9639a6f8b5732a819878f Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 14:22:19 +0800 Subject: [PATCH 47/78] add defaultParallelism to options --- core/src/main/java/com/dtstack/flink/sql/Main.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 415b6e2a9..54ba2cde2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -111,6 +111,8 @@ public static void main(String[] args) throws Exception { options.addOption("savePointPath", true, "Savepoint restore path"); options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); + options.addOption("defaultParallelism",false,"defaultParallelism"); + CommandLineParser parser = new DefaultParser(); CommandLine cl = parser.parse(options, args); From c56c0ad1da166201ba87eee6fa3a4e6750ded921 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 19:45:34 +0800 Subject: [PATCH 48/78] add serversocket source --- docs/serversocketSource.md | 73 +++++++ .../flink/sql/source/kafka/KafkaSource.java | 1 + pom.xml | 1 + serversocket/pom.xml | 40 ++++ serversocket/serversocket-source/pom.xml | 83 ++++++++ .../CustomerSocketTextStreamFunction.java | 179 ++++++++++++++++++ .../serversocket/ServersocketSource.java | 62 ++++++ .../table/ServersocketSourceParser.java | 49 +++++ .../table/ServersocketSourceTableInfo.java | 101 ++++++++++ 9 files changed, 589 insertions(+) create mode 100644 docs/serversocketSource.md create mode 100644 serversocket/pom.xml create mode 100644 serversocket/serversocket-source/pom.xml create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java diff --git a/docs/serversocketSource.md b/docs/serversocketSource.md new file mode 100644 index 000000000..cffdc8dd2 --- /dev/null +++ b/docs/serversocketSource.md @@ -0,0 +1,73 @@ + +## 1.数据格式: +``` +数据现在只支持json格式 {"xx":"bb","cc":"dd"} + +CREATE TABLE MyTable( + channel varchar, + pv int, + xctime date, + xtime date + + )WITH( + type='serversocket', + host='127.0.0.1', + port='8888', + delimiter=';', + maxNumRetries='100' + ); +``` + + +## 2.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | serversocket | 是|| +|host | server host|是|| +|port | server port|是|| +|delimiter| 每条json数据的分割符(比如:;)|是|| +|maxNumRetries| 最大重连次数 (大于0)|是|| + + +## 3.Server端样例: +``` +String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; + + +public class TimeServerHandler implements Runnable { + Socket socket; + + String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; + + public TimeServerHandler(Socket socket) { + this.socket = socket; + } + + public void run() { + PrintWriter out = null; + try { + out = new PrintWriter(this.socket.getOutputStream(), true); + while (true) { + Thread.sleep(3000); + out.println(str); + } + } catch (IOException e) { + e.printStackTrace(); + + if (out != null) { + out.close(); + } + if (socket != null) { + try { + socket.close(); + } catch (IOException e1) { + e1.printStackTrace(); + } + } + } catch (InterruptedException e) { + e.printStackTrace(); + } + } +} +``` 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 129802fbc..02a6bd0b4 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 @@ -35,6 +35,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; diff --git a/pom.xml b/pom.xml index 58cf34f03..de0f1f7d6 100644 --- a/pom.xml +++ b/pom.xml @@ -24,6 +24,7 @@ oracle cassandra kafka08 + serversocket diff --git a/serversocket/pom.xml b/serversocket/pom.xml new file mode 100644 index 000000000..dd76df619 --- /dev/null +++ b/serversocket/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.serversocket + 1.0-SNAPSHOT + + serversocket-source + + pom + + + 3.8.1 + 1.0-SNAPSHOT + + + + + junit + junit + ${junit.version} + test + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + \ No newline at end of file diff --git a/serversocket/serversocket-source/pom.xml b/serversocket/serversocket-source/pom.xml new file mode 100644 index 000000000..4df2c3849 --- /dev/null +++ b/serversocket/serversocket-source/pom.xml @@ -0,0 +1,83 @@ + + + + sql.serversocket + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.source.serversocket + 1.0-SNAPSHOT + jar + + serversocket-source + http://maven.apache.org + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + 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/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java new file mode 100644 index 000000000..8e6004e90 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -0,0 +1,179 @@ +/** + * 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.source.serversocket; + +import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.types.Row; +import org.apache.flink.util.IOUtils; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Iterator; + + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerSocketTextStreamFunction implements SourceFunction { + + /** + * Default delay between successive connection attempts. + */ + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + + /** + * Default connection timeout when connecting to the server socket (infinite). + */ + private static final int CONNECTION_TIMEOUT_TIME = 0; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** + * Type information describing the result type. + */ + private final TypeInformation typeInfo; + + /** + * Field names to parse. Indices match fieldTypes indices. + */ + private final String[] fieldNames; + + /** + * Types to parse fields as. Indices match fieldNames indices. + */ + private final TypeInformation[] fieldTypes; + + private volatile boolean isRunning = true; + + private transient Socket currentSocket; + + ServersocketSourceTableInfo tableInfo; + + public CustomerSocketTextStreamFunction(ServersocketSourceTableInfo tableInfo, TypeInformation typeInfo) { + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.tableInfo = tableInfo; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final StringBuilder buffer = new StringBuilder(); + long attempt = 0; + + while (isRunning) { + + try (Socket socket = new Socket()) { + currentSocket = socket; + + socket.connect(new InetSocketAddress(tableInfo.getHostname(), tableInfo.getPort()), CONNECTION_TIMEOUT_TIME); + try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { + + char[] cbuf = new char[8192]; + int bytesRead; + while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { + buffer.append(cbuf, 0, bytesRead); + int delimPos; + String delimiter = tableInfo.getDelimiter(); + while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { + String record = buffer.substring(0, delimPos); + // truncate trailing carriage return + if (delimiter.equals("\n") && record.endsWith("\r")) { + record = record.substring(0, record.length() - 1); + } + ctx.collect(convertToRow(record)); + buffer.delete(0, delimPos + delimiter.length()); + } + } + } + } + + // if we dropped out of this loop due to an EOF, sleep and retry + if (isRunning) { + attempt++; + if (tableInfo.getMaxNumRetries() == -1 || attempt < tableInfo.getMaxNumRetries()) { + Thread.sleep(DEFAULT_CONNECTION_RETRY_SLEEP); + } else { + // this should probably be here, but some examples expect simple exists of the stream source + // throw new EOFException("Reached end of stream and reconnects are not enabled."); + break; + } + } + } + + // collect trailing data + if (buffer.length() > 0) { + ctx.collect(convertToRow(buffer.toString())); + } + } + + public Row convertToRow(String record) throws IOException { + JsonNode root = objectMapper.readTree(record); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + if (node == null) { + row.setField(i, null); + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + return row; + } + + + @Override + public void cancel() { + isRunning = false; + + // we need to close the socket as well, because the Thread.interrupt() function will + // not wake the thread in the socketStream.read() method when blocked. + Socket theSocket = this.currentSocket; + if (theSocket != null) { + IOUtils.closeSocket(theSocket); + } + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + return null; + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java new file mode 100644 index 000000000..3a67d2c98 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java @@ -0,0 +1,62 @@ +/** + * 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.source.serversocket; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +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.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSource implements IStreamSourceGener

{ + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + ServersocketSourceTableInfo serversocketSourceTableInfo = (ServersocketSourceTableInfo) sourceTableInfo; + + String tableName = serversocketSourceTableInfo.getName(); + + TypeInformation[] types = new TypeInformation[serversocketSourceTableInfo.getFields().length]; + for (int i = 0; i < serversocketSourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(serversocketSourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, serversocketSourceTableInfo.getFields()); + + String fields = StringUtils.join(serversocketSourceTableInfo.getFields(), ","); + + CustomerSocketTextStreamFunction customerSocketTextStreamFunction = new CustomerSocketTextStreamFunction(serversocketSourceTableInfo, typeInformation); + + DataStreamSource serversocketSource = env.addSource(customerSocketTextStreamFunction, tableName, typeInformation); + + return tableEnv.fromDataStream(serversocketSource, fields); + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java new file mode 100644 index 000000000..b52e38e59 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java @@ -0,0 +1,49 @@ +/** + * 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.source.serversocket.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSourceParser extends AbsSourceParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + ServersocketSourceTableInfo serversocketSourceTableInfo = new ServersocketSourceTableInfo(); + serversocketSourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, serversocketSourceTableInfo); + + serversocketSourceTableInfo.setHostname(MathUtil.getString(props.get(ServersocketSourceTableInfo.HOSTNAME_KEY.toLowerCase()))); + serversocketSourceTableInfo.setPort(MathUtil.getIntegerVal(props.get(ServersocketSourceTableInfo.PORT_KEY.toLowerCase()))); + serversocketSourceTableInfo.setDelimiter(MathUtil.getString(props.get(ServersocketSourceTableInfo.DELIMITER_KEY.toLowerCase()))); + serversocketSourceTableInfo.setMaxNumRetries(MathUtil.getLongVal(props.get(ServersocketSourceTableInfo.MAXNUMRETRIES_KEY.toLowerCase()))); + + serversocketSourceTableInfo.check(); + + return serversocketSourceTableInfo; + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java new file mode 100644 index 000000000..2ff9a6739 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java @@ -0,0 +1,101 @@ +/** + * 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.source.serversocket.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSourceTableInfo extends SourceTableInfo { + //version + private static final String CURR_TYPE = "serversocket"; + + public static final String HOSTNAME_KEY = "host"; + + public static final String PORT_KEY = "port"; + + public static final String DELIMITER_KEY = "delimiter"; + + public static final String MAXNUMRETRIES_KEY = "maxNumRetries"; + + + public ServersocketSourceTableInfo() { + super.setType(CURR_TYPE); + } + + private String hostname; + + private int port; + + private String delimiter; + + private long maxNumRetries; + + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + public String getDelimiter() { + return delimiter; + } + + public void setDelimiter(String delimiter) { + this.delimiter = delimiter; + } + + public long getMaxNumRetries() { + return maxNumRetries; + } + + public void setMaxNumRetries(long maxNumRetries) { + this.maxNumRetries = maxNumRetries; + } + + + @Override + public boolean check() { + Preconditions.checkNotNull(hostname,"host name not null"); + Preconditions.checkNotNull(port,"port not null"); + Preconditions.checkNotNull(delimiter,"delimiter name not null"); + Preconditions.checkNotNull(maxNumRetries,"maxNumRetries name not null"); + + Preconditions.checkArgument(port > 0 && port < 65536, "port is out of range"); + Preconditions.checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + return false; + } + + +} From 94bc99ffff42ffe743f85d125d0d57b74e105c85 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 19 Dec 2018 11:29:49 +0800 Subject: [PATCH 49/78] code optimize --- .../main/java/com/dtstack/flink/sql/Main.java | 70 ++++++++----------- .../MyLocalStreamEnvironment.java | 2 +- .../sql/options}/LauncherOptionParser.java | 11 +-- .../flink/sql/options}/LauncherOptions.java | 7 +- .../sql/launcher/ClusterClientFactory.java | 3 +- .../flink/sql/launcher/LauncherMain.java | 2 +- .../sql/launcher/perjob/PerJobSubmitter.java | 2 +- 7 files changed, 39 insertions(+), 58 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{ => environment}/MyLocalStreamEnvironment.java (99%) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql/options}/LauncherOptionParser.java (96%) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql/options}/LauncherOptions.java (94%) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 54ba2cde2..5ff2c6450 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -22,6 +22,9 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; +import com.dtstack.flink.sql.options.LauncherOptionParser; +import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.parser.*; import com.dtstack.flink.sql.side.SideSqlExec; import com.dtstack.flink.sql.side.SideTableInfo; @@ -42,6 +45,7 @@ import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -59,7 +63,6 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,44 +102,22 @@ public class Main { private static final int delayInterval = 10; //sec public static void main(String[] args) throws Exception { - - Options options = new Options(); - options.addOption("sql", true, "sql config"); - options.addOption("name", true, "job name"); - options.addOption("addjar", true, "add jar"); - options.addOption("localSqlPluginPath", true, "local sql plugin path"); - options.addOption("remoteSqlPluginPath", true, "remote sql plugin path"); - options.addOption("confProp", true, "env properties"); - options.addOption("mode", true, "deploy mode"); - - options.addOption("savePointPath", true, "Savepoint restore path"); - options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); - options.addOption("defaultParallelism",false,"defaultParallelism"); - - - CommandLineParser parser = new DefaultParser(); - CommandLine cl = parser.parse(options, args); - String sql = cl.getOptionValue("sql"); - String name = cl.getOptionValue("name"); - String addJarListStr = cl.getOptionValue("addjar"); - String localSqlPluginPath = cl.getOptionValue("localSqlPluginPath"); - String remoteSqlPluginPath = cl.getOptionValue("remoteSqlPluginPath"); - String deployMode = cl.getOptionValue("mode"); - String confProp = cl.getOptionValue("confProp"); - - Preconditions.checkNotNull(sql, "parameters of sql is required"); - Preconditions.checkNotNull(name, "parameters of name is required"); - Preconditions.checkNotNull(localSqlPluginPath, "parameters of localSqlPluginPath is required"); - + LauncherOptionParser optionParser = new LauncherOptionParser(args); + LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String sql = launcherOptions.getSql(); + String name =launcherOptions.getName(); + String addJarListStr = launcherOptions.getAddjar(); + String localSqlPluginPath = launcherOptions.getLocalSqlPluginPath(); + String remoteSqlPluginPath = launcherOptions.getRemoteSqlPluginPath(); + String deployMode = launcherOptions.getMode(); + String confProp = launcherOptions.getConfProp(); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); - List addJarFileList = Lists.newArrayList(); if(!Strings.isNullOrEmpty(addJarListStr)){ addJarListStr = URLDecoder.decode(addJarListStr, Charsets.UTF_8.name()); addJarFileList = objMapper.readValue(addJarListStr, List.class); } - ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); Thread.currentThread().setContextClassLoader(dtClassLoader); @@ -240,6 +221,12 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla contextEnvironment.getClasspaths().add(url); } } + int i = 0; + for(URL url : classPathSet){ + String classFileName = String.format(CLASS_FILE_NAME_FMT, i); + env.registerCachedFile(url.getPath(), classFileName, true); + i++; + } } private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, @@ -291,18 +278,23 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); registerTableCache.put(tableInfo.getName(), regTable); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + } } else if (tableInfo instanceof TargetTableInfo) { TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + } } else if(tableInfo instanceof SideTableInfo){ - String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + } }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } @@ -310,12 +302,6 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en //The plug-in information corresponding to the table is loaded into the classPath env addEnvClassPath(env, classPathSet); - int i = 0; - for(URL url : classPathSet){ - String classFileName = String.format(CLASS_FILE_NAME_FMT, i); - env.registerCachedFile(url.getPath(), classFileName, true); - i++; - } } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java similarity index 99% rename from core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java rename to core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java index acd28e3f3..c1cea1e14 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql; +package com.dtstack.flink.sql.environment; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java similarity index 96% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java rename to core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java index a801b193e..d4b4a19f2 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql.options; import avro.shaded.com.google.common.collect.Lists; import org.apache.commons.cli.BasicParser; @@ -73,7 +73,7 @@ public class LauncherOptionParser { private LauncherOptions properties = new LauncherOptions(); - public LauncherOptionParser(String[] args) { + public LauncherOptionParser(String[] args) throws Exception { options.addOption(OPTION_MODE, true, "Running mode"); options.addOption(OPTION_SQL, true, "Job sql file"); options.addOption(OPTION_NAME, true, "Job name"); @@ -87,8 +87,6 @@ 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); String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); //check mode @@ -109,7 +107,6 @@ public LauncherOptionParser(String[] args) { properties.setLocalSqlPluginPath(localPlugin); String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - Preconditions.checkNotNull(remotePlugin); properties.setRemoteSqlPluginPath(remotePlugin); String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); @@ -148,10 +145,6 @@ public LauncherOptionParser(String[] args) { if(StringUtils.isNotBlank(flinkJarPath)){ properties.setFlinkJarPath(flinkJarPath); } - - } catch (Exception e) { - throw new RuntimeException(e); - } } public LauncherOptions getLauncherOptions(){ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java similarity index 94% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java rename to core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java index d9c5bd1ed..07c349fda 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java @@ -16,7 +16,10 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql.options; + +import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; /** * This class define commandline options for the Launcher program @@ -26,7 +29,7 @@ */ public class LauncherOptions { - private String mode; + private String mode = ClusterMode.local.name(); private String name; 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 02af5fff4..be8611757 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 @@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; -import java.io.FilenameFilter; import java.net.InetSocketAddress; import java.net.URLDecoder; import java.util.*; @@ -52,7 +51,7 @@ import java.io.IOException; import java.util.stream.Collectors; import java.util.stream.Stream; - +import com.dtstack.flink.sql.options.LauncherOptions; import static java.util.Objects.requireNonNull; /** 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 ec6a3e10c..f32ed9008 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 @@ -32,7 +32,6 @@ 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; @@ -41,6 +40,7 @@ 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; +import com.dtstack.flink.sql.options.*; /** * Date: 2017/2/20 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 index 3cdf3f8d4..5c0032ce6 100644 --- 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 @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.launcher.perjob; -import com.dtstack.flink.sql.launcher.LauncherOptions; +import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; From b6d24736585950bef30614c5dc246d3185578a14 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 19 Dec 2018 18:56:36 +0800 Subject: [PATCH 50/78] async cache get data order or unorder --- .../dtstack/flink/sql/side/SideTableInfo.java | 12 ++++++++++++ .../sql/side/operator/SideAsyncOperator.java | 15 +++++++++++++-- .../flink/sql/table/AbsSideTableParser.java | 16 ++++++++++++---- docs/mysqlSide.md | 2 ++ 4 files changed, 39 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 0abd55a92..02c9b4e02 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -45,6 +45,8 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String PARTITIONED_JOIN_KEY = "partitionedJoin"; + public static final String CACHE_MODE_KEY = "cacheMode"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; @@ -53,6 +55,8 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { private boolean partitionedJoin = false; + private String cacheMode="ordered"; + public RowTypeInfo getRowTypeInfo(){ Class[] fieldClass = getFieldClasses(); TypeInformation[] types = new TypeInformation[fieldClass.length]; @@ -95,4 +99,12 @@ public boolean isPartitionedJoin() { public void setPartitionedJoin(boolean partitionedJoin) { this.partitionedJoin = partitionedJoin; } + + public String getCacheMode() { + return cacheMode; + } + + public void setCacheMode(String cacheMode) { + this.cacheMode = cacheMode; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index df9a1c175..95121faa8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -25,6 +25,7 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; +import com.sun.org.apache.bcel.internal.generic.I2F; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; @@ -44,6 +45,9 @@ public class SideAsyncOperator { private static final String PATH_FORMAT = "%sasyncside"; + private static final String ORDERED = "ordered"; + + //TODO need to set by create table task private static int asyncCapacity = 100; @@ -62,8 +66,15 @@ private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowT public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { AsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) - .setParallelism(sideTableInfo.getParallelism()); + if (ORDERED.equals(sideTableInfo.getCacheMode())){ + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + .setParallelism(sideTableInfo.getParallelism()); + }else { + return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + .setParallelism(sideTableInfo.getParallelism()); + } + } } 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 f8ede801b..26305f7d6 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 @@ -16,13 +16,14 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.table; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.MathUtil; +import org.apache.commons.lang3.StringUtils; import java.util.Map; import java.util.regex.Matcher; @@ -65,9 +66,7 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setCacheType(cacheType); if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); - if(cacheSize < 0){ - throw new RuntimeException("cache size need > 0."); - } + sideTableInfo.setCacheSize(cacheSize); } @@ -85,6 +84,15 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setPartitionedJoin(true); } } + + if(props.containsKey(SideTableInfo.CACHE_MODE_KEY.toLowerCase())){ + String cachemode = MathUtil.getString(props.get(SideTableInfo.CACHE_MODE_KEY.toLowerCase())); + + if(!cachemode.equalsIgnoreCase("ordered") && !cachemode.equalsIgnoreCase("unordered")){ + throw new RuntimeException("cachemode must ordered or unordered!"); + } + sideTableInfo.setCacheMode(cachemode.toLowerCase()); + } } } } diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index d0fec5832..63e745776 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -52,6 +52,7 @@ * LRU: * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认无序。 ## 5.样例 @@ -70,6 +71,7 @@ create table sideTable( cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', + cacheMode='unordered', parallelism ='1', partitionedJoin='false' ); From 10a3ebbb58355449da9da55cd40acea6ebcc394a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 19 Dec 2018 19:03:59 +0800 Subject: [PATCH 51/78] modify default cachemode order --- .../dtstack/flink/sql/side/operator/SideAsyncOperator.java | 1 - .../java/com/dtstack/flink/sql/table/AbsSideTableParser.java | 4 +++- docs/mysqlSide.md | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 95121faa8..121d557bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -25,7 +25,6 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; -import com.sun.org.apache.bcel.internal.generic.I2F; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; 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 26305f7d6..526a632dd 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 @@ -66,7 +66,9 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setCacheType(cacheType); if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); - + if(cacheSize < 0){ + throw new RuntimeException("cache size need > 0."); + } sideTableInfo.setCacheSize(cacheSize); } diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index 63e745776..c6f5d741a 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -52,7 +52,7 @@ * LRU: * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) - * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认无序。 + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 ## 5.样例 From e9cc74a27b39cf4a081cbcd49adaf4cf0f8ccefd Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 20 Dec 2018 12:36:56 +0800 Subject: [PATCH 52/78] add console sink --- README.md | 2 +- console/console-sink/pom.xml | 79 ++++ .../sql/sink/console/ConsoleOutputFormat.java | 118 ++++++ .../flink/sql/sink/console/ConsoleSink.java | 86 +++++ .../sink/console/table/ConsoleSinkParser.java | 45 +++ .../sink/console/table/ConsoleTableInfo.java | 46 +++ .../sink/console/table/TablePrintUtil.java | 336 ++++++++++++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++ console/pom.xml | 34 ++ docs/consoleSink.md | 50 +++ pom.xml | 1 + 11 files changed, 854 insertions(+), 1 deletion(-) create mode 100644 console/console-sink/pom.xml create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java create mode 100644 console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 console/pom.xml create mode 100644 docs/consoleSink.md diff --git a/README.md b/README.md index ad14222ca..e2056dd17 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,7 @@ # 已支持 * 源表:kafka 0.9,1.x版本 * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra + * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 * 增加kafka结果表功能 diff --git a/console/console-sink/pom.xml b/console/console-sink/pom.xml new file mode 100644 index 000000000..a5ca2629f --- /dev/null +++ b/console/console-sink/pom.xml @@ -0,0 +1,79 @@ + + + + sql.console + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + console-sink + jar + + console-sink + http://maven.apache.org + + + + 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/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java new file mode 100644 index 000000000..55bed93be --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java @@ -0,0 +1,118 @@ +/** + * 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.console; + +import com.dtstack.flink.sql.sink.MetricOutputFormat; +import com.dtstack.flink.sql.sink.console.table.TablePrintUtil; +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.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleOutputFormat extends MetricOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleOutputFormat.class); + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + @Override + public void configure(Configuration parameters) { + + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + initMetric(); + } + + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + if (!retract) { + //FIXME 暂时不处理Mongo删除操作--->Mongo要求有key,所有认为都是可以执行update查找 + return; + } + + Row record = tupleTrans.getField(1); + if (record.getArity() != fieldNames.length) { + return; + } + + List data = new ArrayList<>(); + data.add(fieldNames); + data.add(record.toString().split(",")); + TablePrintUtil.build(data).print(); + + outRecords.inc(); + } + + @Override + public void close() throws IOException { + + } + + private ConsoleOutputFormat() { + } + + public static ConsoleOutputFormatBuilder buildOutputFormat() { + return new ConsoleOutputFormatBuilder(); + } + + public static class ConsoleOutputFormatBuilder { + + private final ConsoleOutputFormat format; + + protected ConsoleOutputFormatBuilder() { + this.format = new ConsoleOutputFormat(); + } + + public ConsoleOutputFormatBuilder setFieldNames(String[] fieldNames) { + format.fieldNames = fieldNames; + return this; + } + + public ConsoleOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + format.fieldTypes = fieldTypes; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractConsoleCOutputFormat + */ + public ConsoleOutputFormat finish() { + return format; + } + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java new file mode 100644 index 000000000..77a3efea2 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java @@ -0,0 +1,86 @@ +/** + * 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.console; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +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.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; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + @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; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + ConsoleOutputFormat.ConsoleOutputFormatBuilder builder = ConsoleOutputFormat.buildOutputFormat(); + builder.setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + ConsoleOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public ConsoleSink genStreamSink(TargetTableInfo targetTableInfo) { + return this; + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java new file mode 100644 index 000000000..e77444bfd --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java @@ -0,0 +1,45 @@ +/** + * 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.console.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; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + ConsoleTableInfo consoleTableInfo = new ConsoleTableInfo(); + consoleTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, consoleTableInfo); + + consoleTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + return consoleTableInfo; + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java new file mode 100644 index 000000000..4b286c667 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java @@ -0,0 +1,46 @@ +/** + * 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.console.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "console"; + + public ConsoleTableInfo() { + setType(CURR_TYPE); + } + + @Override + public boolean check() { + return true; + } + + @Override + public String getType() { + return super.getType().toLowerCase(); + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java new file mode 100644 index 000000000..8934d4394 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -0,0 +1,336 @@ +package com.dtstack.flink.sql.sink.console.table; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class TablePrintUtil { + public static final int ALIGN_LEFT = 1;//左对齐 + public static final int ALIGN_RIGHT = 2;//右对齐 + public static final int ALIGN_CENTER = 3;//居中对齐 + + private int align = ALIGN_CENTER;//默认居中对齐 + private boolean equilong = false;//默认不等宽 + private int padding = 1;//左右边距默认为1 + private char h = '-';//默认水平分隔符 + private char v = '|';//默认竖直分隔符 + private char o = '+';//默认交叉分隔符 + private char s = ' ';//默认空白填充符 + private List data;//数据 + + private TablePrintUtil() { + } + + /** + * 链式调用入口方法 + * + * @param data + * @return + */ + public static TablePrintUtil build(String[][] data) { + TablePrintUtil self = new TablePrintUtil(); + self.data = new ArrayList<>(Arrays.asList(data)); + return self; + } + + /** + * 链式调用入口方法,T可以是String[]、List、任意实体类 + * 由于java泛型不同无法重载,所以这里要写if instanceof进行类型判断 + * + * @param data + * @param + * @return + */ + public static TablePrintUtil build(List data) { + TablePrintUtil self = new TablePrintUtil(); + self.data = new ArrayList<>(); + if (data.size() <= 0) throw new RuntimeException("数据源至少得有一行吧"); + Object obj = data.get(0); + + + if (obj instanceof String[]) { + //如果泛型为String数组,则直接设置 + self.data = (List) data; + } else if (obj instanceof List) { + //如果泛型为List,则把list中的item依次转为String[],再设置 + int length = ((List) obj).size(); + for (Object item : data) { + List col = (List) item; + if (col.size() != length) throw new RuntimeException("数据源每列长度必须一致"); + self.data.add(col.toArray(new String[length])); + } + } else { + //如果泛型为实体类,则利用反射获取get方法列表,从而推算出属性列表。 + //根据反射得来的属性列表设置表格第一行thead + List

colList = getColList(obj); + String[] header = new String[colList.size()]; + for (int i = 0; i < colList.size(); i++) { + header[i] = colList.get(i).colName; + } + self.data.add(header); + //利用反射调用相应get方法获取属性值来设置表格tbody + for (int i = 0; i < data.size(); i++) { + String[] item = new String[colList.size()]; + for (int j = 0; j < colList.size(); j++) { + String value = null; + try { + value = obj.getClass().getMethod(colList.get(j).getMethodName).invoke(data.get(i)).toString(); + } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + e.printStackTrace(); + } + item[j] = value == null ? "null" : value; + } + self.data.add(item); + } + } + return self; + } + + private static class Col { + private String colName;//列名 + private String getMethodName;//get方法名 + } + + /** + * 利用反射获取get方法名和属性名 + * + * @return + */ + private static ListgetColList(Object obj) { + ListcolList = new ArrayList<>(); + Method[] methods = obj.getClass().getMethods(); + for (Method m : methods) { + StringBuilder getMethodName = new StringBuilder(m.getName()); + if (getMethodName.substring(0, 3).equals("get") && !m.getName().equals("getClass")) { + Col col = new Col(); + col.getMethodName = getMethodName.toString(); + char first = Character.toLowerCase(getMethodName.delete(0, 3).charAt(0)); + getMethodName.delete(0, 1).insert(0, first); + col.colName = getMethodName.toString(); + colList.add(col); + } + } + return colList; + } + + /** + * 获取字符串占的字符位数 + * + * @param str + * @return + */ + private int getStringCharLength(String str) { + Pattern p = Pattern.compile("[\u4e00-\u9fa5]");//利用正则找到中文 + Matcher m = p.matcher(str); + int count = 0; + while (m.find()) { + count++; + } + return str.length() + count; + } + + /** + * 纵向遍历获取数据每列的长度 + * + * @return + */ + private int[] getColLengths() { + int[] result = new int[data.get(0).length]; + for (int x = 0; x < result.length; x++) { + int max = 0; + for (int y = 0; y < data.size(); y++) { + int len = getStringCharLength(data.get(y)[x]); + if (len > max) { + max = len; + } + } + result[x] = max; + } + if (equilong) {//如果等宽表格 + int max = 0; + for (int len : result) { + if (len > max) max = len; + } + for (int i = 0; i < result.length; i++) { + result[i] = max; + } + } + return result; + } + + /** + * 取得表格字符串 + * + * @return + */ + public String getTableString() { + StringBuilder sb = new StringBuilder(); + int[] colLengths = getColLengths();//获取每列文字宽度 + StringBuilder line = new StringBuilder();//表格横向分隔线 + line.append(o); + for (int len : colLengths) { + int allLen = len + padding * 2;//还需要加上边距和分隔符的长度 + for (int i = 0; i < allLen; i++) { + line.append(h); + } + line.append(o); + } + sb.append(line).append("\r\n"); + for (int y = 0; y < data.size(); y++) { + sb.append(v); + for (int x = 0; x < data.get(y).length; x++) { + String cell = data.get(y)[x]; + switch (align) { + case ALIGN_LEFT: + for (int i = 0; i < padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + break; + case ALIGN_RIGHT: + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < padding; i++) sb.append(s); + break; + case ALIGN_CENTER: + int space = colLengths[x] - getStringCharLength(cell); + int left = space / 2; + int right = space - left; + for (int i = 0; i < left + padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < right + padding; i++) sb.append(s); + break; + } + sb.append(v); + } + sb.append("\r\n"); + sb.append(line).append("\r\n"); + } + return sb.toString(); + } + + /** + * 直接打印表格 + */ + public void print() { + System.out.println(getTableString()); + } + + //下面是链式调用的set方法 + public TablePrintUtil setAlign(int align) { + this.align = align; + return this; + } + + public TablePrintUtil setEquilong(boolean equilong) { + this.equilong = equilong; + return this; + } + + public TablePrintUtil setPadding(int padding) { + this.padding = padding; + return this; + } + + public TablePrintUtil setH(char h) { + this.h = h; + return this; + } + + public TablePrintUtil setV(char v) { + this.v = v; + return this; + } + + public TablePrintUtil setO(char o) { + this.o = o; + return this; + } + + public TablePrintUtil setS(char s) { + this.s = s; + return this; + } + + /** + * 使用示例 + * + * @param args + */ + public static void main(String[] args) { + List data1 = new ArrayList<>(); + data1.add(new String[]{"用户名", "密码", "姓名"}); + data1.add(new String[]{"xiaoming", "xm123", "小明"}); + data1.add(new String[]{"xiaohong", "xh123", "小红"}); + TablePrintUtil.build(data1).print(); + + List> data2 = new ArrayList<>(); + data2.add(new ArrayList<>()); + data2.add(new ArrayList<>()); + data2.add(new ArrayList<>()); + data2.get(0).add("用户名"); + data2.get(0).add("密码"); + data2.get(0).add("姓名"); + data2.get(1).add("xiaoming"); + data2.get(1).add("xm123"); + data2.get(1).add("小明"); + data2.get(2).add("xiaohong"); + data2.get(2).add("xh123"); + data2.get(2).add("小红"); + TablePrintUtil.build(data2) + .setAlign(TablePrintUtil.ALIGN_LEFT) + .setPadding(5) + .setEquilong(true) + .print(); + + + class User { + String username; + String password; + String name; + + User(String username, String password, String name) { + this.username = username; + this.password = password; + this.name = name; + } + + 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; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + } + List data3 = new ArrayList<>(); + data3.add(new User("xiaoming", "xm123", "小明")); + data3.add(new User("xiaohong", "xh123", "小红")); + TablePrintUtil.build(data3).setH('=').setV('!').print(); + } +} diff --git a/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..e03e5451f --- /dev/null +++ b/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.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.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest(String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/console/pom.xml b/console/pom.xml new file mode 100644 index 000000000..983e1c185 --- /dev/null +++ b/console/pom.xml @@ -0,0 +1,34 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.console + pom + + + console-sink + + + + + junit + junit + 3.8.1 + test + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/docs/consoleSink.md b/docs/consoleSink.md new file mode 100644 index 000000000..206d7faaa --- /dev/null +++ b/docs/consoleSink.md @@ -0,0 +1,50 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='console', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 +没有限制 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型[console]|是|| +| parallelism | 并行度设置|否|1| + +## 5.样例: +``` +CREATE TABLE MyResult( + name VARCHAR, + channel VARCHAR + )WITH( + type ='console', + parallelism ='1' + ) + ``` + + ## 6.输出结果: + ``` + +------+---------+ + | name | channel | + +------+---------+ + | aa | 02 | + +------+---------+ + ``` \ No newline at end of file diff --git a/pom.xml b/pom.xml index de0f1f7d6..c3ad4d24b 100644 --- a/pom.xml +++ b/pom.xml @@ -25,6 +25,7 @@ cassandra kafka08 serversocket + console From fad6c3a1d45c1b3d94732c820af5c317eee4c34e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 20 Dec 2018 13:57:54 +0800 Subject: [PATCH 53/78] code --- .../flink/sql/sink/console/ConsoleOutputFormat.java | 1 - .../flink/sql/sink/console/table/TablePrintUtil.java | 12 ++++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java index 55bed93be..7658e9979 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java @@ -59,7 +59,6 @@ public void writeRecord(Tuple2 tuple2) throws IOException { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); if (!retract) { - //FIXME 暂时不处理Mongo删除操作--->Mongo要求有key,所有认为都是可以执行update查找 return; } diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java index 8934d4394..b9f42386b 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -192,22 +192,22 @@ public String getTableString() { String cell = data.get(y)[x]; switch (align) { case ALIGN_LEFT: - for (int i = 0; i < padding; i++) sb.append(s); + for (int i = 0; i < padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) {sb.append(s);} break; case ALIGN_RIGHT: - for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < padding; i++) sb.append(s); + for (int i = 0; i < padding; i++) {sb.append(s);} break; case ALIGN_CENTER: int space = colLengths[x] - getStringCharLength(cell); int left = space / 2; int right = space - left; - for (int i = 0; i < left + padding; i++) sb.append(s); + for (int i = 0; i < left + padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < right + padding; i++) sb.append(s); + for (int i = 0; i < right + padding; i++) {sb.append(s);} break; } sb.append(v); From 83e7ddc74b292fb755bd24b321dcf37276e57711 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Dec 2018 19:33:49 +0800 Subject: [PATCH 54/78] fix kafka offset bug --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4daf83d6f..339cf6264 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 93e7d7642..6ce03a19b 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index b151a93e9..693b8668d 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -124,7 +124,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From 14f1b0823a8cb1e3be230679e67840174e2f1240 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Dec 2018 21:30:55 +0800 Subject: [PATCH 55/78] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=B3=A8=E9=87=8A?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/ClusterMode.java | 4 ++++ core/src/main/java/com/dtstack/flink/sql/Main.java | 5 +++++ .../com/dtstack/flink/sql/parser/CreateFuncParser.java | 2 +- .../com/dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../dtstack/flink/sql/parser/CreateTmpTableParser.java | 6 ++++++ .../com/dtstack/flink/sql/parser/InsertSqlParser.java | 3 +-- .../main/java/com/dtstack/flink/sql/parser/SqlTree.java | 2 +- .../java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 - .../flink/sql/side/operator/SideAsyncOperator.java | 4 +++- .../flink/sql/side/operator/SideWithAllCacheOperator.java | 4 +++- .../java/com/dtstack/flink/sql/table/TableInfoParser.java | 1 - .../sql/source/kafka/table/KafkaSourceTableInfo.java | 8 ++++---- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 8 ++++---- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 6 +++--- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 7 ++++--- 15 files changed, 40 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java index 024a31854..82105d9e7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java @@ -31,4 +31,8 @@ public enum ClusterMode { ClusterMode(int type){ this.type = type; } + + public int getType(){ + return this.type; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 5ff2c6450..01f4474a0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -102,8 +102,10 @@ public class Main { private static final int delayInterval = 10; //sec public static void main(String[] args) throws Exception { + LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String sql = launcherOptions.getSql(); String name =launcherOptions.getName(); String addJarListStr = launcherOptions.getAddjar(); @@ -111,13 +113,16 @@ public static void main(String[] args) throws Exception { String remoteSqlPluginPath = launcherOptions.getRemoteSqlPluginPath(); String deployMode = launcherOptions.getMode(); String confProp = launcherOptions.getConfProp(); + sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); List addJarFileList = Lists.newArrayList(); + if(!Strings.isNullOrEmpty(addJarListStr)){ addJarListStr = URLDecoder.decode(addJarListStr, Charsets.UTF_8.name()); addJarFileList = objMapper.readValue(addJarListStr, List.class); } + ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); Thread.currentThread().setContextClassLoader(dtClassLoader); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java index 9b7017743..793dd6baa 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java @@ -24,7 +24,7 @@ import java.util.regex.Pattern; /** - * 解析创建自定义方法sql + * parser register udf sql * Date: 2018/6/26 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index b5e4a4aa4..5e126e786 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -29,7 +29,7 @@ import java.util.regex.Pattern; /** - * 解析创建表结构sql + * parser create table sql * Date: 2018/6/26 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 5840b26a8..db18986b7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -33,6 +33,12 @@ import static org.apache.calcite.sql.SqlKind.IDENTIFIER; +/** + * parser create tmp table sql + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author yanxi + */ public class CreateTmpTableParser implements IParser { //select table tableName as select diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index f62b09d04..40629b139 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -32,8 +32,7 @@ import static org.apache.calcite.sql.SqlKind.IDENTIFIER; /** - * 解析flink sql - * sql 只支持 insert 开头的 + * parser flink sql * Date: 2018/6/22 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java index 3ed37c51e..754de0819 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -29,7 +29,7 @@ import java.util.Map; /** - * 解析sql获得的对象结构 + * parser sql to get the Sql Tree structure * Date: 2018/6/25 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 388bb5497..b412b1d18 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 121d557bf..29f0fe2e4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -44,6 +44,8 @@ public class SideAsyncOperator { private static final String PATH_FORMAT = "%sasyncside"; + private static final String OPERATOR_TYPE = "Async"; + private static final String ORDERED = "ordered"; @@ -57,7 +59,7 @@ private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowT String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side", "Async"); + String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java index 72a67d00b..725798848 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java @@ -43,6 +43,8 @@ public class SideWithAllCacheOperator { private static final String PATH_FORMAT = "%sallside"; + private static final String OPERATOR_TYPE = "All"; + private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { @@ -53,7 +55,7 @@ private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowType DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side", "All"); + String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); return dtClassLoader.loadClass(className).asSubclass(AllReqRow.class).getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java index b3a07d6d5..4c2f67f68 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.table; -import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.parser.CreateTableParser; import com.dtstack.flink.sql.side.SideTableInfo; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4fb59f7d2..b59bb8055 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -52,7 +52,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); @@ -70,9 +70,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 874e2c3b9..5e875f48b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -59,7 +59,7 @@ public KafkaSourceTableInfo(){ } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ @@ -81,9 +81,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 3184f4df3..3aedce2c7 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -82,9 +82,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index ffc270486..942104732 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -56,7 +56,7 @@ public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key, String value) { kafkaParam.put(key, value); @@ -74,8 +74,9 @@ public Set getKafkaParamKeys() { public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From ba29e96d9591c107924861f82536e43691512619 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 20 Dec 2018 23:47:12 +0800 Subject: [PATCH 56/78] change the doc --- README.md | 2 ++ docs/{serversocketSource.md => serverSocketSource.md} | 0 2 files changed, 2 insertions(+) rename docs/{serversocketSource.md => serverSocketSource.md} (100%) diff --git a/README.md b/README.md index e2056dd17..808b40432 100644 --- a/README.md +++ b/README.md @@ -143,6 +143,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ## 2 结构 ### 2.1 源表插件 * [kafka 源表插件](docs/kafkaSource.md) +* [serverSocket 源表插件](docs/serverSocketSource.md) ### 2.2 结果表插件 * [elasticsearch 结果表插件](docs/elasticsearchSink.md) @@ -151,6 +152,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) * [cassandra 结果表插件](docs/cassandraSink.md) +* [console 结果表插件](docs/consoleSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) diff --git a/docs/serversocketSource.md b/docs/serverSocketSource.md similarity index 100% rename from docs/serversocketSource.md rename to docs/serverSocketSource.md From d69121e05ecd3d26ed3ab431a4eca17593c209aa Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 21 Dec 2018 10:48:04 +0800 Subject: [PATCH 57/78] code optimese --- .../java/com/dtstack/flink/sql/launcher/LauncherMain.java | 7 ------- 1 file changed, 7 deletions(-) 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 f32ed9008..f4c715093 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 @@ -84,13 +84,6 @@ public static void main(String[] args) throws Exception { PerJobSubmitter.submit(launcherOptions, jobGraph); } else { ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - pluginRoot = launcherOptions.getLocalSqlPluginPath(); - jarFile = new File(getLocalCoreJarPath(pluginRoot)); - remoteArgs = argList.toArray(new String[argList.size()]); - 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, launcherOptions.getDefaultParallelism()); clusterClient.shutdown(); System.exit(0); From 009c971965ff720750735a8962af31b2bfdf3d5a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 17:51:02 +0800 Subject: [PATCH 58/78] add timezone --- .../flink/sql/table/SourceTableInfo.java | 29 + .../CustomerWaterMarkerForLong.java | 22 +- .../CustomerWaterMarkerForTimeStamp.java | 18 +- .../sql/watermarker/WaterMarkerAssigner.java | 9 +- docs/kafkaSource.md | 3 + docs/timeZone.md | 601 ++++++++++++++++++ .../source/kafka/table/KafkaSourceParser.java | 2 + .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 3 +- .../flink/sql/launcher/LauncherMain.java | 4 +- 11 files changed, 680 insertions(+), 13 deletions(-) create mode 100644 docs/timeZone.md diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java index d4282a968..ebe675c75 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -20,10 +20,14 @@ package com.dtstack.flink.sql.table; +import com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import java.util.ArrayList; import java.util.Map; +import java.util.TimeZone; /** * Reason: @@ -38,8 +42,12 @@ public abstract class SourceTableInfo extends TableInfo { public static final String SOURCE_DATA_TYPE = "sourcedatatype"; + public static final String TIME_ZONE_KEY="timezone"; + private String sourceDataType = "json"; + private String timeZone="Asia/Shanghai"; + private String eventTimeField; private Integer maxOutOrderness = 10; @@ -113,4 +121,25 @@ public String getSourceDataType() { public void setSourceDataType(String sourceDataType) { this.sourceDataType = sourceDataType; } + + public String getTimeZone() { + return timeZone; + } + + public void setTimeZone(String timeZone) { + if (timeZone==null){ + return; + } + timeZoneCheck(timeZone); + this.timeZone = timeZone; + } + + private void timeZoneCheck(String timeZone) { + ArrayList zones = Lists.newArrayList(TimeZone.getAvailableIDs()); + if (!zones.contains(timeZone)){ + throw new IllegalArgumentException(" timezone is Incorrect!"); + } + } + + } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 0a6874a81..7a466f118 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.watermarker; @@ -27,6 +27,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.TimeZone; + /** * Custom watermark --- for eventtime * Date: 2017/12/28 @@ -40,13 +42,16 @@ public class CustomerWaterMarkerForLong extends AbsCustomerWaterMarker { private static final long serialVersionUID = 1L; + private TimeZone timezone; + private int pos; private long lastTime = 0; - public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos) { + public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override @@ -55,12 +60,21 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); lastTime = eveTime; - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - eveTime)/1000)); - return eveTime; + + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(eveTime))/1000)); + + long restime=lastTime + timezone.getOffset(lastTime); + + return restime; }catch (Exception e){ logger.error("", e); } return lastTime; } + + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index b4aee0302..779f837ff 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.TimeZone; /** * Custom watermark --- for eventtime @@ -45,10 +46,13 @@ public class CustomerWaterMarkerForTimeStamp extends AbsCustomerWaterMarker private long lastTime = 0; + private TimeZone timezone; - public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos) { + + public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override @@ -57,13 +61,19 @@ public long extractTimestamp(Row row) { Timestamp time = (Timestamp) row.getField(pos); lastTime = time.getTime(); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - time.getTime())/1000)); - return time.getTime(); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(lastTime))/1000)); + + long restime=lastTime + timezone.getOffset(lastTime); + + return restime; } catch (RuntimeException e) { logger.error("", e); } return lastTime; } - + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index a29e8391b..4f386d75f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -31,6 +32,8 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.util.TimeZone; + /** * define watermarker * Date: 2018/6/29 @@ -54,6 +57,8 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type int maxOutOrderness = sourceTableInfo.getMaxOutOrderness(); + String timeZone=sourceTableInfo.getTimeZone(); + String[] fieldNames = typeInfo.getFieldNames(); TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); @@ -75,9 +80,9 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type AbsCustomerWaterMarker waterMarker = null; if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.sql.Timestamp")){ - waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos,timeZone); }else if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.lang.Long")){ - waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos,timeZone); }else{ throw new IllegalArgumentException("not support type of " + fieldType + ", current only support(timestamp, long)."); } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 2f6ce202c..a78239421 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -15,6 +15,8 @@ CREATE TABLE tableName( kafka.auto.offset.reset ='latest', kafka.topic ='topicName', parallelism ='parllNum', + --timezone='America/Los_Angeles', + timezone='Asia/Shanghai', sourcedatatype ='json' #可不设置 ); ``` @@ -46,6 +48,7 @@ CREATE TABLE tableName( |kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| |sourcedatatype | 数据类型|否|json| +|timezone|时区设置[timezone支持的参数](timeZone.md)|否|'Asia/Shanghai' **kafka相关参数可以自定义,使用kafka.开头即可。** ## 5.样例: diff --git a/docs/timeZone.md b/docs/timeZone.md new file mode 100644 index 000000000..ec6aa0ca3 --- /dev/null +++ b/docs/timeZone.md @@ -0,0 +1,601 @@ +* Africa/Abidjan +* Africa/Accra +* Africa/Addis_Ababa +* Africa/Algiers +* Africa/Asmara +* Africa/Asmera +* Africa/Bamako +* Africa/Bangui +* Africa/Banjul +* Africa/Bissau +* Africa/Blantyre +* Africa/Brazzaville +* Africa/Bujumbura +* Africa/Cairo +* Africa/Casablanca +* Africa/Ceuta +* Africa/Conakry +* Africa/Dakar +* Africa/Dar_es_Salaam +* Africa/Djibouti +* Africa/Douala +* Africa/El_Aaiun +* Africa/Freetown +* Africa/Gaborone +* Africa/Harare +* Africa/Johannesburg +* Africa/Juba +* Africa/Kampala +* Africa/Khartoum +* Africa/Kigali +* Africa/Kinshasa +* Africa/Lagos +* Africa/Libreville +* Africa/Lome +* Africa/Luanda +* Africa/Lubumbashi +* Africa/Lusaka +* Africa/Malabo +* Africa/Maputo +* Africa/Maseru +* Africa/Mbabane +* Africa/Mogadishu +* Africa/Monrovia +* Africa/Nairobi +* Africa/Ndjamena +* Africa/Niamey +* Africa/Nouakchott +* Africa/Ouagadougou +* Africa/Porto-Novo +* Africa/Sao_Tome +* Africa/Timbuktu +* Africa/Tripoli +* Africa/Tunis +* Africa/Windhoek +* America/Adak +* America/Anchorage +* America/Anguilla +* America/Antigua +* America/Araguaina +* America/Argentina/Buenos_Aires +* America/Argentina/Catamarca +* America/Argentina/ComodRivadavia +* America/Argentina/Cordoba +* America/Argentina/Jujuy +* America/Argentina/La_Rioja +* America/Argentina/Mendoza +* America/Argentina/Rio_Gallegos +* America/Argentina/Salta +* America/Argentina/San_Juan +* America/Argentina/San_Luis +* America/Argentina/Tucuman +* America/Argentina/Ushuaia +* America/Aruba +* America/Asuncion +* America/Atikokan +* America/Atka +* America/Bahia +* America/Bahia_Banderas +* America/Barbados +* America/Belem +* America/Belize +* America/Blanc-Sablon +* America/Boa_Vista +* America/Bogota +* America/Boise +* America/Buenos_Aires +* America/Cambridge_Bay +* America/Campo_Grande +* America/Cancun +* America/Caracas +* America/Catamarca +* America/Cayenne +* America/Cayman +* America/Chicago +* America/Chihuahua +* America/Coral_Harbour +* America/Cordoba +* America/Costa_Rica +* America/Creston +* America/Cuiaba +* America/Curacao +* America/Danmarkshavn +* America/Dawson +* America/Dawson_Creek +* America/Denver +* America/Detroit +* America/Dominica +* America/Edmonton +* America/Eirunepe +* America/El_Salvador +* America/Ensenada +* America/Fort_Nelson +* America/Fort_Wayne +* America/Fortaleza +* America/Glace_Bay +* America/Godthab +* America/Goose_Bay +* America/Grand_Turk +* America/Grenada +* America/Guadeloupe +* America/Guatemala +* America/Guayaquil +* America/Guyana +* America/Halifax +* America/Havana +* America/Hermosillo +* America/Indiana/Indianapolis +* America/Indiana/Knox +* America/Indiana/Marengo +* America/Indiana/Petersburg +* America/Indiana/Tell_City +* America/Indiana/Vevay +* America/Indiana/Vincennes +* America/Indiana/Winamac +* America/Indianapolis +* America/Inuvik +* America/Iqaluit +* America/Jamaica +* America/Jujuy +* America/Juneau +* America/Kentucky/Louisville +* America/Kentucky/Monticello +* America/Knox_IN +* America/Kralendijk +* America/La_Paz +* America/Lima +* America/Los_Angeles +* America/Louisville +* America/Lower_Princes +* America/Maceio +* America/Managua +* America/Manaus +* America/Marigot +* America/Martinique +* America/Matamoros +* America/Mazatlan +* America/Mendoza +* America/Menominee +* America/Merida +* America/Metlakatla +* America/Mexico_City +* America/Miquelon +* America/Moncton +* America/Monterrey +* America/Montevideo +* America/Montreal +* America/Montserrat +* America/Nassau +* America/New_York +* America/Nipigon +* America/Nome +* America/Noronha +* America/North_Dakota/Beulah +* America/North_Dakota/Center +* America/North_Dakota/New_Salem +* America/Ojinaga +* America/Panama +* America/Pangnirtung +* America/Paramaribo +* America/Phoenix +* America/Port-au-Prince +* America/Port_of_Spain +* America/Porto_Acre +* America/Porto_Velho +* America/Puerto_Rico +* America/Punta_Arenas +* America/Rainy_River +* America/Rankin_Inlet +* America/Recife +* America/Regina +* America/Resolute +* America/Rio_Branco +* America/Rosario +* America/Santa_Isabel +* America/Santarem +* America/Santiago +* America/Santo_Domingo +* America/Sao_Paulo +* America/Scoresbysund +* America/Shiprock +* America/Sitka +* America/St_Barthelemy +* America/St_Johns +* America/St_Kitts +* America/St_Lucia +* America/St_Thomas +* America/St_Vincent +* America/Swift_Current +* America/Tegucigalpa +* America/Thule +* America/Thunder_Bay +* America/Tijuana +* America/Toronto +* America/Tortola +* America/Vancouver +* America/Virgin +* America/Whitehorse +* America/Winnipeg +* America/Yakutat +* America/Yellowknife +* Antarctica/Casey +* Antarctica/Davis +* Antarctica/DumontDUrville +* Antarctica/Macquarie +* Antarctica/Mawson +* Antarctica/McMurdo +* Antarctica/Palmer +* Antarctica/Rothera +* Antarctica/South_Pole +* Antarctica/Syowa +* Antarctica/Troll +* Antarctica/Vostok +* Arctic/Longyearbyen +* Asia/Aden +* Asia/Almaty +* Asia/Amman +* Asia/Anadyr +* Asia/Aqtau +* Asia/Aqtobe +* Asia/Ashgabat +* Asia/Ashkhabad +* Asia/Atyrau +* Asia/Baghdad +* Asia/Bahrain +* Asia/Baku +* Asia/Bangkok +* Asia/Barnaul +* Asia/Beirut +* Asia/Bishkek +* Asia/Brunei +* Asia/Calcutta +* Asia/Chita +* Asia/Choibalsan +* Asia/Chongqing +* Asia/Chungking +* Asia/Colombo +* Asia/Dacca +* Asia/Damascus +* Asia/Dhaka +* Asia/Dili +* Asia/Dubai +* Asia/Dushanbe +* Asia/Famagusta +* Asia/Gaza +* Asia/Harbin +* Asia/Hebron +* Asia/Ho_Chi_Minh +* Asia/Hong_Kong +* Asia/Hovd +* Asia/Irkutsk +* Asia/Istanbul +* Asia/Jakarta +* Asia/Jayapura +* Asia/Jerusalem +* Asia/Kabul +* Asia/Kamchatka +* Asia/Karachi +* Asia/Kashgar +* Asia/Kathmandu +* Asia/Katmandu +* Asia/Khandyga +* Asia/Kolkata +* Asia/Krasnoyarsk +* Asia/Kuala_Lumpur +* Asia/Kuching +* Asia/Kuwait +* Asia/Macao +* Asia/Macau +* Asia/Magadan +* Asia/Makassar +* Asia/Manila +* Asia/Muscat +* Asia/Nicosia +* Asia/Novokuznetsk +* Asia/Novosibirsk +* Asia/Omsk +* Asia/Oral +* Asia/Phnom_Penh +* Asia/Pontianak +* Asia/Pyongyang +* Asia/Qatar +* Asia/Qyzylorda +* Asia/Rangoon +* Asia/Riyadh +* Asia/Saigon +* Asia/Sakhalin +* Asia/Samarkand +* Asia/Seoul +* Asia/Shanghai +* Asia/Singapore +* Asia/Srednekolymsk +* Asia/Taipei +* Asia/Tashkent +* Asia/Tbilisi +* Asia/Tehran +* Asia/Tel_Aviv +* Asia/Thimbu +* Asia/Thimphu +* Asia/Tokyo +* Asia/Tomsk +* Asia/Ujung_Pandang +* Asia/Ulaanbaatar +* Asia/Ulan_Bator +* Asia/Urumqi +* Asia/Ust-Nera +* Asia/Vientiane +* Asia/Vladivostok +* Asia/Yakutsk +* Asia/Yangon +* Asia/Yekaterinburg +* Asia/Yerevan +* Atlantic/Azores +* Atlantic/Bermuda +* Atlantic/Canary +* Atlantic/Cape_Verde +* Atlantic/Faeroe +* Atlantic/Faroe +* Atlantic/Jan_Mayen +* Atlantic/Madeira +* Atlantic/Reykjavik +* Atlantic/South_Georgia +* Atlantic/St_Helena +* Atlantic/Stanley +* Australia/ACT +* Australia/Adelaide +* Australia/Brisbane +* Australia/Broken_Hill +* Australia/Canberra +* Australia/Currie +* Australia/Darwin +* Australia/Eucla +* Australia/Hobart +* Australia/LHI +* Australia/Lindeman +* Australia/Lord_Howe +* Australia/Melbourne +* Australia/NSW +* Australia/North +* Australia/Perth +* Australia/Queensland +* Australia/South +* Australia/Sydney +* Australia/Tasmania +* Australia/Victoria +* Australia/West +* Australia/Yancowinna +* Brazil/Acre +* Brazil/DeNoronha +* Brazil/East +* Brazil/West +* CET +* CST6CDT +* Canada/Atlantic +* Canada/Central +* Canada/Eastern +* Canada/Mountain +* Canada/Newfoundland +* Canada/Pacific +* Canada/Saskatchewan +* Canada/Yukon +* Chile/Continental +* Chile/EasterIsland +* Cuba +* EET +* EST5EDT +* Egypt +* Eire +* Etc/GMT +* Etc/GMT+0 +* Etc/GMT+1 +* Etc/GMT+10 +* Etc/GMT+11 +* Etc/GMT+12 +* Etc/GMT+2 +* Etc/GMT+3 +* Etc/GMT+4 +* Etc/GMT+5 +* Etc/GMT+6 +* Etc/GMT+7 +* Etc/GMT+8 +* Etc/GMT+9 +* Etc/GMT-0 +* Etc/GMT-1 +* Etc/GMT-10 +* Etc/GMT-11 +* Etc/GMT-12 +* Etc/GMT-13 +* Etc/GMT-14 +* Etc/GMT-2 +* Etc/GMT-3 +* Etc/GMT-4 +* Etc/GMT-5 +* Etc/GMT-6 +* Etc/GMT-7 +* Etc/GMT-8 +* Etc/GMT-9 +* Etc/GMT0 +* Etc/Greenwich +* Etc/UCT +* Etc/UTC +* Etc/Universal +* Etc/Zulu +* Europe/Amsterdam +* Europe/Andorra +* Europe/Astrakhan +* Europe/Athens +* Europe/Belfast +* Europe/Belgrade +* Europe/Berlin +* Europe/Bratislava +* Europe/Brussels +* Europe/Bucharest +* Europe/Budapest +* Europe/Busingen +* Europe/Chisinau +* Europe/Copenhagen +* Europe/Dublin +* Europe/Gibraltar +* Europe/Guernsey +* Europe/Helsinki +* Europe/Isle_of_Man +* Europe/Istanbul +* Europe/Jersey +* Europe/Kaliningrad +* Europe/Kiev +* Europe/Kirov +* Europe/Lisbon +* Europe/Ljubljana +* Europe/London +* Europe/Luxembourg +* Europe/Madrid +* Europe/Malta +* Europe/Mariehamn +* Europe/Minsk +* Europe/Monaco +* Europe/Moscow +* Europe/Nicosia +* Europe/Oslo +* Europe/Paris +* Europe/Podgorica +* Europe/Prague +* Europe/Riga +* Europe/Rome +* Europe/Samara +* Europe/San_Marino +* Europe/Sarajevo +* Europe/Saratov +* Europe/Simferopol +* Europe/Skopje +* Europe/Sofia +* Europe/Stockholm +* Europe/Tallinn +* Europe/Tirane +* Europe/Tiraspol +* Europe/Ulyanovsk +* Europe/Uzhgorod +* Europe/Vaduz +* Europe/Vatican +* Europe/Vienna +* Europe/Vilnius +* Europe/Volgograd +* Europe/Warsaw +* Europe/Zagreb +* Europe/Zaporozhye +* Europe/Zurich +* GB +* GB-Eire +* GMT +* GMT0 +* Greenwich +* Hongkong +* Iceland +* Indian/Antananarivo +* Indian/Chagos +* Indian/Christmas +* Indian/Cocos +* Indian/Comoro +* Indian/Kerguelen +* Indian/Mahe +* Indian/Maldives +* Indian/Mauritius +* Indian/Mayotte +* Indian/Reunion +* Iran +* Israel +* Jamaica +* Japan +* Kwajalein +* Libya +* MET +* MST7MDT +* Mexico/BajaNorte +* Mexico/BajaSur +* Mexico/General +* NZ +* NZ-CHAT +* Navajo +* PRC +* PST8PDT +* Pacific/Apia +* Pacific/Auckland +* Pacific/Bougainville +* Pacific/Chatham +* Pacific/Chuuk +* Pacific/Easter +* Pacific/Efate +* Pacific/Enderbury +* Pacific/Fakaofo +* Pacific/Fiji +* Pacific/Funafuti +* Pacific/Galapagos +* Pacific/Gambier +* Pacific/Guadalcanal +* Pacific/Guam +* Pacific/Honolulu +* Pacific/Johnston +* Pacific/Kiritimati +* Pacific/Kosrae +* Pacific/Kwajalein +* Pacific/Majuro +* Pacific/Marquesas +* Pacific/Midway +* Pacific/Nauru +* Pacific/Niue +* Pacific/Norfolk +* Pacific/Noumea +* Pacific/Pago_Pago +* Pacific/Palau +* Pacific/Pitcairn +* Pacific/Pohnpei +* Pacific/Ponape +* Pacific/Port_Moresby +* Pacific/Rarotonga +* Pacific/Saipan +* Pacific/Samoa +* Pacific/Tahiti +* Pacific/Tarawa +* Pacific/Tongatapu +* Pacific/Truk +* Pacific/Wake +* Pacific/Wallis +* Pacific/Yap +* Poland +* Portugal +* ROK +* Singapore +* SystemV/AST4 +* SystemV/AST4ADT +* SystemV/CST6 +* SystemV/CST6CDT +* SystemV/EST5 +* SystemV/EST5EDT +* SystemV/HST10 +* SystemV/MST7 +* SystemV/MST7MDT +* SystemV/PST8 +* SystemV/PST8PDT +* SystemV/YST9 +* SystemV/YST9YDT +* Turkey +* UCT +* US/Alaska +* US/Aleutian +* US/Arizona +* US/Central +* US/East-Indiana +* US/Eastern +* US/Hawaii +* US/Indiana-Starke +* US/Michigan +* US/Mountain +* US/Pacific +* US/Pacific-New +* US/Samoa +* UTC +* Universal +* W-SU +* WET +* Zulu + + diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 9af3c88ca..eb085ac78 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -46,6 +46,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map argList = optionParser.getProgramExeArgList(); if(mode.equals(ClusterMode.local.name())) { - String[] localArgs = argList.toArray(new String[argList.size()]); - Main.main(localArgs); + //String[] localArgs = argList.toArray(new String[argList.size()]); + Main.main(args); }else{ String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); From 987eea876c9b7c6f54cfc6140c5f8772b34f48e9 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 18:13:49 +0800 Subject: [PATCH 59/78] fix watermark lasttime --- .../sql/watermarker/CustomerWaterMarkerForLong.java | 10 ++++------ .../watermarker/CustomerWaterMarkerForTimeStamp.java | 11 ++++++----- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 7a466f118..57143a5ea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -59,17 +59,15 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); - lastTime = eveTime; + Long extractTime=eveTime; - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(eveTime))/1000)); + lastTime = extractTime + timezone.getOffset(extractTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); - long restime=lastTime + timezone.getOffset(lastTime); - - return restime; + return lastTime; }catch (Exception e){ logger.error("", e); } - return lastTime; } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index 779f837ff..e4af63744 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.watermarker; @@ -59,13 +59,14 @@ public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos,String ti public long extractTimestamp(Row row) { try { Timestamp time = (Timestamp) row.getField(pos); - lastTime = time.getTime(); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(lastTime))/1000)); + long extractTime=time.getTime(); + + lastTime = extractTime + timezone.getOffset(extractTime); - long restime=lastTime + timezone.getOffset(lastTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); - return restime; + return lastTime; } catch (RuntimeException e) { logger.error("", e); } From a7000cdcdcf498b74b9f15dae25e6dfcfa6247e2 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 22:19:13 +0800 Subject: [PATCH 60/78] side async param --- .../dtstack/flink/sql/side/SideTableInfo.java | 24 +++++++++++++++++++ .../sql/side/operator/SideAsyncOperator.java | 7 ++---- .../flink/sql/table/AbsSideTableParser.java | 16 +++++++++++++ .../CustomerWaterMarkerForLong.java | 1 + docs/mysqlSide.md | 5 +++- 5 files changed, 47 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 02c9b4e02..e21389ea7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -47,12 +47,20 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String CACHE_MODE_KEY = "cacheMode"; + public static final String ASYNC_CAP_KEY = "asyncCapacity"; + + public static final String ASYNC_TIMEOUT_KEY = "asyncTimeout"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; private long cacheTimeout = 60 * 1000;// + private int asyncCapacity=100; + + private int asyncTimeout=10000; + private boolean partitionedJoin = false; private String cacheMode="ordered"; @@ -107,4 +115,20 @@ public String getCacheMode() { public void setCacheMode(String cacheMode) { this.cacheMode = cacheMode; } + + public int getAsyncCapacity() { + return asyncCapacity; + } + + public void setAsyncCapacity(int asyncCapacity) { + this.asyncCapacity = asyncCapacity; + } + + public int getAsyncTimeout() { + return asyncTimeout; + } + + public void setAsyncTimeout(int asyncTimeout) { + this.asyncTimeout = asyncTimeout; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 29f0fe2e4..688a19c35 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -49,9 +49,6 @@ public class SideAsyncOperator { private static final String ORDERED = "ordered"; - //TODO need to set by create table task - private static int asyncCapacity = 100; - private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); @@ -70,10 +67,10 @@ public static DataStream getSideJoinDataStream(DataStream inputStream, String si //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? if (ORDERED.equals(sideTableInfo.getCacheMode())){ - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); }else { - return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); } 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 526a632dd..ae8135256 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 @@ -95,6 +95,22 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p } sideTableInfo.setCacheMode(cachemode.toLowerCase()); } + + if(props.containsKey(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())){ + Integer asyncCap = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())); + if(asyncCap < 0){ + throw new RuntimeException("asyncCapacity size need > 0."); + } + sideTableInfo.setAsyncCapacity(asyncCap); + } + + if(props.containsKey(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())){ + Integer asyncTimeout = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())); + if (asyncTimeout<0){ + throw new RuntimeException("asyncTimeout size need > 0."); + } + sideTableInfo.setAsyncTimeout(asyncTimeout); + } } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 57143a5ea..632c10c7b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -62,6 +62,7 @@ public long extractTimestamp(Row row) { Long extractTime=eveTime; lastTime = extractTime + timezone.getOffset(extractTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); return lastTime; diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index c6f5d741a..b17c72bb6 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -53,7 +53,8 @@ * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 - + * asyncCapacity:异步请求容量,默认1000 + * asyncTimeout:异步请求超时时间,默认10000毫秒 ## 5.样例 ``` @@ -72,6 +73,8 @@ create table sideTable( cacheSize ='10000', cacheTTLMs ='60000', cacheMode='unordered', + asyncCapacity='1000', + asyncTimeout='10000' parallelism ='1', partitionedJoin='false' ); From 05f10bc6308d0249ce7115dc5029a06e9d2548e8 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 24 Dec 2018 18:39:30 +0800 Subject: [PATCH 61/78] fix asyn rdb write only one record --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) 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 73fab0e9f..6909b312a 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 @@ -90,11 +90,14 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except return; } else if (ECacheContentType.MultiLine == val.getType()) { + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } + resultFuture.complete(rowList); + } else { throw new RuntimeException("not support cache obj type " + val.getType()); } @@ -122,18 +125,18 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except int resultSize = rs.result().getResults().size(); if (resultSize > 0) { + List rowList = Lists.newArrayList(); for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); if (openCache()) { cacheContent.add(line); } - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - if (openCache()) { putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } + resultFuture.complete(rowList); } else { dealMissKey(input, resultFuture); if (openCache()) { From 740f0b5f5a08629a6a6d88a91cc8ba58b861d486 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 25 Dec 2018 11:54:28 +0800 Subject: [PATCH 62/78] fix parse bug --- core/src/main/java/com/dtstack/flink/sql/Main.java | 5 ----- .../java/com/dtstack/flink/sql/launcher/LauncherMain.java | 4 +--- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 01f4474a0..9c71fce1c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -40,17 +40,12 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.DefaultParser; -import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; 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 4467d1a44..47718581f 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 @@ -67,13 +67,11 @@ public static void main(String[] args) throws Exception { String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); if(mode.equals(ClusterMode.local.name())) { - //String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(args); }else{ 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); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), args); if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); } From 8c7546972e1b631cd3b4eefb361a42a1de765649 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 25 Dec 2018 18:02:15 +0800 Subject: [PATCH 63/78] console sink add log and serversocketSource modify exception scope --- .../sink/console/table/TablePrintUtil.java | 5 +++ .../CustomerSocketTextStreamFunction.java | 44 ++++++++++--------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java index b9f42386b..8813da619 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -1,5 +1,8 @@ package com.dtstack.flink.sql.sink.console.table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; @@ -15,6 +18,7 @@ * @author xuqianjin */ public class TablePrintUtil { + private static final Logger LOG = LoggerFactory.getLogger(TablePrintUtil.class); public static final int ALIGN_LEFT = 1;//左对齐 public static final int ALIGN_RIGHT = 2;//右对齐 public static final int ALIGN_CENTER = 3;//居中对齐 @@ -222,6 +226,7 @@ public String getTableString() { * 直接打印表格 */ public void print() { + LOG.info("\n"+getTableString()); System.out.println(getTableString()); } diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java index 8e6004e90..ce9a44895 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -25,6 +25,8 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.IOException; @@ -42,11 +44,12 @@ * @author maqi */ public class CustomerSocketTextStreamFunction implements SourceFunction { + private static final Logger LOG = LoggerFactory.getLogger(CustomerSocketTextStreamFunction.class); /** * Default delay between successive connection attempts. */ - private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 2000; /** * Default connection timeout when connecting to the server socket (infinite). @@ -92,32 +95,33 @@ public void run(SourceContext ctx) throws Exception { long attempt = 0; while (isRunning) { - - try (Socket socket = new Socket()) { + try { + Socket socket = new Socket(); currentSocket = socket; - socket.connect(new InetSocketAddress(tableInfo.getHostname(), tableInfo.getPort()), CONNECTION_TIMEOUT_TIME); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { - - char[] cbuf = new char[8192]; - int bytesRead; - while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { - buffer.append(cbuf, 0, bytesRead); - int delimPos; - String delimiter = tableInfo.getDelimiter(); - while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { - String record = buffer.substring(0, delimPos); - // truncate trailing carriage return - if (delimiter.equals("\n") && record.endsWith("\r")) { - record = record.substring(0, record.length() - 1); - } - ctx.collect(convertToRow(record)); - buffer.delete(0, delimPos + delimiter.length()); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + char[] cbuf = new char[8192]; + int bytesRead; + while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { + buffer.append(cbuf, 0, bytesRead); + int delimPos; + String delimiter = tableInfo.getDelimiter(); + while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { + String record = buffer.substring(0, delimPos); + // truncate trailing carriage return + if (delimiter.equals("\n") && record.endsWith("\r")) { + record = record.substring(0, record.length() - 1); } + ctx.collect(convertToRow(record)); + buffer.delete(0, delimPos + delimiter.length()); } } + } catch (Exception e) { + LOG.info("Connection server failed, Please check configuration !!!!!!!!!!!!!!!!"); } + // if we dropped out of this loop due to an EOF, sleep and retry if (isRunning) { attempt++; From 916da8b4e81acd8570e2efe77be62662ab6264c4 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 26 Dec 2018 16:23:43 +0800 Subject: [PATCH 64/78] fix kafka cvs pattern error and remove kafka08 monitor --- kafka08/kafka08-source/pom.xml | 2 +- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../sql/source/kafka/consumer/CustomerCsvConsumer.java | 9 --------- .../sql/source/kafka/consumer/CustomerJsonConsumer.java | 9 --------- .../deserialization/CustomerCommonDeserialization.java | 8 ++++---- .../deserialization/CustomerCsvDeserialization.java | 8 ++++---- .../deserialization/CustomerJsonDeserialization.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- 10 files changed, 29 insertions(+), 47 deletions(-) diff --git a/kafka08/kafka08-source/pom.xml b/kafka08/kafka08-source/pom.xml index df98c2603..64264fd46 100644 --- a/kafka08/kafka08-source/pom.xml +++ b/kafka08/kafka08-source/pom.xml @@ -20,7 +20,7 @@ org.apache.flink flink-connector-kafka-0.8_2.11 - 1.6.0 + ${flink.version} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 2e7136666..bfbffdf14 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -90,19 +90,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization(typeInformation, kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); } else { - kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index ad859e485..7dc95450e 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -20,20 +20,11 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 0b775a4dc..b627d81c0 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -20,20 +20,11 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index f35bacce7..c92ce4aa4 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -48,9 +48,9 @@ public class CustomerCommonDeserialization extends AbsDeserialization imple @Override public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { - numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); + //numInRecord.inc(); + //numInBytes.inc(message.length); + //numInBytes.inc(messageKey.length); try { Row row = Row.of( @@ -63,7 +63,7 @@ public Row deserialize(byte[] messageKey, byte[] message, String topic, int part return row; } catch (Throwable t) { LOG.error(t.getMessage()); - dirtyDataCounter.inc(); + // dirtyDataCounter.inc(); return null; } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index e67d79400..0ca57e0b3 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -80,8 +80,8 @@ public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDel public Row deserialize(byte[] message) throws IOException { try { - numInRecord.inc(); - numInBytes.inc(message.length); + //numInRecord.inc(); + //numInBytes.inc(message.length); String[] fieldsList = null; if (message != null && message.length > 0){ fieldsList = new String(message).split(fieldDelimiter); @@ -103,11 +103,11 @@ public Row deserialize(byte[] message) throws IOException { } } - numInResolveRecord.inc(); + //numInResolveRecord.inc(); return row; } catch (Throwable t) { //add metric of dirty data - dirtyDataCounter.inc(); + //dirtyDataCounter.inc(); throw new RuntimeException(t); } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 72b8b2c8e..900717b98 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -80,8 +80,8 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ public Row deserialize(byte[] message) throws IOException { try { - numInRecord.inc(); - numInBytes.inc(message.length); + // numInRecord.inc(); + // numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); @@ -102,12 +102,12 @@ public Row deserialize(byte[] message) throws IOException { } } - numInResolveRecord.inc(); + // numInResolveRecord.inc(); return row; } catch (Throwable t) { //add metric of dirty data LOG.error(t.getMessage()); - dirtyDataCounter.inc(); + // dirtyDataCounter.inc(); return null; } } 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 c01366e4c..292f70f51 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 @@ -100,19 +100,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } 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 586f046af..ef85e5b56 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 @@ -103,19 +103,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } 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 02a6bd0b4..0f538aa1d 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 @@ -100,19 +100,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } From 4540349ea5d6435ff971228d3cbf5c53fa59271e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 26 Dec 2018 19:18:56 +0800 Subject: [PATCH 65/78] more kafka bugfix --- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 3 +-- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 8 files changed, 8 insertions(+), 9 deletions(-) diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 20e228d30..5dae21742 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -42,7 +42,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index b59bb8055..9f7e36f56 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -52,7 +52,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index bad84231d..48daf57dd 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -40,7 +40,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 5e875f48b..d2f7d212d 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -59,7 +59,7 @@ public KafkaSourceTableInfo(){ } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 6557355d9..0235547d2 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -42,7 +42,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 3aedce2c7..de75c27cb 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -57,8 +57,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); - + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index b6b2dd978..78edf17b4 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -41,7 +41,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 942104732..983a1882a 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -56,7 +56,7 @@ public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key, String value) { kafkaParam.put(key, value); From bf0a3884d0aa84d656dd0a02e603fd5a7169b1a8 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 27 Dec 2018 15:05:09 +0800 Subject: [PATCH 66/78] yarnper submit bugfix --- .../sql/launcher/ClusterClientFactory.java | 2 +- .../sql/launcher/perjob/PerJobSubmitter.java | 26 ++++++++++++------- 2 files changed, 18 insertions(+), 10 deletions(-) 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 be8611757..4513649fa 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 @@ -63,7 +63,7 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarn.name()) || mode.equals(ClusterMode.yarnPer.name())) { + } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions,mode); } throw new IllegalArgumentException("Unsupported cluster client type: "); 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 index 5c0032ce6..7944142a2 100644 --- 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 @@ -20,6 +20,7 @@ import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -27,7 +28,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Map; import java.util.Properties; /** @@ -39,28 +42,33 @@ public class PerJobSubmitter { + private final static String CLASS_FILE_NAME_PRESTR = "class_path"; + 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); - + ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification,fillJobGraphClassPath(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; } + + private static JobGraph fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith(CLASS_FILE_NAME_PRESTR)){ + jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); + } + } + return jobGraph; + } } From 61d81e8df39842881921ded0eb6aa6842393254e Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 8 Jan 2019 20:24:29 +0800 Subject: [PATCH 67/78] remove primary key field space --- .../main/java/com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index a43769918..4fa4f0597 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -109,7 +109,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ } public static void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ - String primaryFields = matcher.group(1); + String primaryFields = matcher.group(1).trim(); String[] splitArry = primaryFields.split(","); List primaryKes = Lists.newArrayList(splitArry); tableInfo.setPrimaryKeys(primaryKes); From 85c27376e1b0c8b46e29901c87ed8efbfebfa92c Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 9 Jan 2019 11:09:24 +0800 Subject: [PATCH 68/78] hbase remove field to upper --- .../java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java | 2 +- .../side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java | 4 ++-- .../sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index de89f337e..c9adc87b9 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -170,7 +170,7 @@ private void loadData(Map> tmpCache) throws SQLExcep StringBuilder key = new StringBuilder(); key.append(family).append(":").append(qualifier); - kv.put(aliasNameInversion.get(key.toString().toUpperCase()), value); + kv.put(aliasNameInversion.get(key.toString()), value); } tmpCache.put(new String(r.getRow()), kv); } diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index d077f3493..d79f67ec5 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -98,9 +98,9 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, String mapKey = cf + ":" + col; //The table format defined using different data type conversion byte - String colType = colRefType.get(mapKey.toUpperCase()); + String colType = colRefType.get(mapKey); Object val = HbaseUtils.convertByte(keyValue.value(), colType); - sideMap.put(mapKey.toUpperCase(), val); + sideMap.put(mapKey, val); } if (oneRow.size() > 0) { diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index a85f58c4b..4a4b60ff6 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -73,9 +73,9 @@ public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFu String col = new String(keyValue.qualifier()); String mapKey = cf + ":" + col; //The table format defined using different data type conversion byte - String colType = colRefType.get(mapKey.toUpperCase()); + String colType = colRefType.get(mapKey); Object val = HbaseUtils.convertByte(keyValue.value(), colType); - sideMap.put(mapKey.toUpperCase(), val); + sideMap.put(mapKey, val); } if(arg.size() > 0){ From b82edb98e8c7bc6c470c64e5a646fdd19cf01605 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 13:54:21 +0800 Subject: [PATCH 69/78] fix mongo read asyncside info bug --- .../com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d376bcdde..d8a22772b 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -142,12 +142,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; } else if (ECacheContentType.MultiLine == val.getType()) { - + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - + resultFuture.complete(rowList); } else { throw new RuntimeException("not support cache obj type " + val.getType()); } From 12bd803ebf9993def741d8e1ee84aa6dc13165f6 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 14:18:33 +0800 Subject: [PATCH 70/78] fix redis async read info bug --- .../dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 1a6aa0425..251107287 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 @@ -141,8 +141,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; }else if(ECacheContentType.MultiLine == val.getType()){ - Row row = fillData(input, val.getContent()); - resultFuture.complete(Collections.singleton(row)); + List rowList = Lists.newArrayList(); + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, val.getContent()); + rowList.add(row); + } + resultFuture.complete(rowList); }else{ throw new RuntimeException("not support cache obj type " + val.getType()); } From 84a44b240e01154d70054ec7aba42d9ae32a63b0 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 17:53:36 +0800 Subject: [PATCH 71/78] mongo rdb syncside bug fix --- .../dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 1 + .../dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d8a22772b..226885ed7 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -130,6 +130,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } basicDBObject.put(sideInfo.getEqualFieldList().get(i), equalObj); } 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 73fab0e9f..35f300c26 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 @@ -75,8 +75,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } - inputParams.add(equalObj); } @@ -89,12 +89,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; } else if (ECacheContentType.MultiLine == val.getType()) { - + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - + resultFuture.complete(rowList); } else { throw new RuntimeException("not support cache obj type " + val.getType()); } From 70b1c778c095cfafb97426dab591f3b519da6464 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 11 Jan 2019 17:24:33 +0800 Subject: [PATCH 72/78] =?UTF-8?q?=E5=B0=86=E6=89=80=E6=9C=89=E5=8F=98?= =?UTF-8?q?=E9=87=8F=E6=B7=BB=E5=8A=A0=E5=88=B0env=E4=B8=8A?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 26 ++++++++++++++++++- .../flink/sql/launcher/LauncherMain.java | 19 ++++++++------ 2 files changed, 36 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 415b6e2a9..37ffa4df4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -42,6 +42,7 @@ import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -52,6 +53,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.configuration.Configuration; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; @@ -67,6 +69,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.net.URLDecoder; @@ -316,12 +319,33 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } } - private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException, NoSuchMethodException { StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); + Configuration globalJobParameters = new Configuration(); + Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); + method.setAccessible(true); + + confProperties.forEach((key,val) -> { + try { + method.invoke(globalJobParameters, key, val); + } catch (IllegalAccessException e) { + e.printStackTrace(); + } catch (InvocationTargetException e) { + e.printStackTrace(); + } + }); + + ExecutionConfig exeConfig = env.getConfig(); + if(exeConfig.getGlobalJobParameters() == null){ + exeConfig.setGlobalJobParameters(globalJobParameters); + }else if(exeConfig.getGlobalJobParameters() instanceof Configuration){ + ((Configuration) exeConfig.getGlobalJobParameters()).addAll(globalJobParameters); + } + if(FlinkUtil.getMaxEnvParallelism(confProperties) > 0){ env.setMaxParallelism(FlinkUtil.getMaxEnvParallelism(confProperties)); 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 eea52e0af..32813393d 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 @@ -23,24 +23,27 @@ import avro.shaded.com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; +import com.dtstack.flink.sql.ClusterMode; 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.*; -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; 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; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; /** * Date: 2017/2/20 From 075064a11d0f2087cf1f5c9820f789fe34e094ff Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 11 Jan 2019 20:57:21 +0800 Subject: [PATCH 73/78] fix rdb asynside bug --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 35f300c26..6989dd64f 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 @@ -26,19 +26,15 @@ import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 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.math.BigInteger; import java.sql.Timestamp; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -122,18 +118,21 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except int resultSize = rs.result().getResults().size(); if (resultSize > 0) { - for (JsonArray line : rs.result().getResults()) { + List rowList = Lists.newArrayList(); + for (JsonArray line : rs.result().getResults()) { Row row = fillData(input, line); if (openCache()) { cacheContent.add(line); } - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } if (openCache()) { putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } + + resultFuture.complete(rowList); } else { dealMissKey(input, resultFuture); if (openCache()) { From 6ea1740e35ecd47142216b20c22669ee85aa4887 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 21 Jan 2019 20:57:30 +0800 Subject: [PATCH 74/78] fix main args parse --- .../main/java/com/dtstack/flink/sql/Main.java | 40 +- .../flink/sql/options/LauncherOptions.java | 9 - .../com/dtstack/flink/yarn/JobParameter.java | 130 -- .../flink/yarn/YarnClusterConfiguration.java | 87 - .../flink/yarn/YarnClusterDescriptor.java | 393 ---- .../calcite/RelTimeIndicatorConverter.scala | 482 ----- .../flink/table/codegen/CodeGenerator.scala | 1920 ----------------- .../table/codegen/MatchCodeGenerator.scala | 574 ----- .../flink/table/codegen/generated.scala | 100 - .../nodes/datastream/DataStreamMatch.scala | 309 --- .../nodes/logical/FlinkLogicalMatch.scala | 115 - .../table/plan/rules/FlinkRuleSets.scala | 209 -- .../datastream/DataStreamMatchRule.scala | 46 - .../table/runtime/cepmatch/ConvertToRow.scala | 14 - .../cepmatch/IterativeConditionRunner.scala | 40 - .../table/runtime/cepmatch/MatchUtil.scala | 99 - .../PatternFlatSelectFunctionRunner.scala | 47 - .../PatternSelectFunctionRunner.scala | 45 - .../table/validate/FunctionCatalog.scala | 534 ----- .../sql/launcher/ClusterClientFactory.java | 207 +- .../flink/sql/launcher/LauncherMain.java | 9 +- 21 files changed, 81 insertions(+), 5328 deletions(-) delete mode 100644 core/src/main/java/com/dtstack/flink/yarn/JobParameter.java delete mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java delete mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java delete mode 100644 core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/generated.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 0bd5b657a..11917c4cd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -37,9 +37,14 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import com.google.common.base.Preconditions; import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.ExecutionConfig; @@ -101,16 +106,31 @@ public class Main { public static void main(String[] args) throws Exception { - LauncherOptionParser optionParser = new LauncherOptionParser(args); - LauncherOptions launcherOptions = optionParser.getLauncherOptions(); - - String sql = launcherOptions.getSql(); - String name =launcherOptions.getName(); - String addJarListStr = launcherOptions.getAddjar(); - String localSqlPluginPath = launcherOptions.getLocalSqlPluginPath(); - String remoteSqlPluginPath = launcherOptions.getRemoteSqlPluginPath(); - String deployMode = launcherOptions.getMode(); - String confProp = launcherOptions.getConfProp(); + Options options = new Options(); + options.addOption("sql", true, "sql config"); + options.addOption("name", true, "job name"); + options.addOption("addjar", true, "add jar"); + options.addOption("localSqlPluginPath", true, "local sql plugin path"); + options.addOption("remoteSqlPluginPath", true, "remote sql plugin path"); + options.addOption("confProp", true, "env properties"); + options.addOption("mode", true, "deploy mode"); + + options.addOption("savePointPath", true, "Savepoint restore path"); + options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); + + CommandLineParser parser = new DefaultParser(); + CommandLine cl = parser.parse(options, args); + String sql = cl.getOptionValue("sql"); + String name = cl.getOptionValue("name"); + String addJarListStr = cl.getOptionValue("addjar"); + String localSqlPluginPath = cl.getOptionValue("localSqlPluginPath"); + String remoteSqlPluginPath = cl.getOptionValue("remoteSqlPluginPath"); + String deployMode = cl.getOptionValue("mode"); + String confProp = cl.getOptionValue("confProp"); + + Preconditions.checkNotNull(sql, "parameters of sql is required"); + Preconditions.checkNotNull(name, "parameters of name is required"); + Preconditions.checkNotNull(localSqlPluginPath, "parameters of localSqlPluginPath is required"); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); diff --git a/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java index 07c349fda..d84637abc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java @@ -56,8 +56,6 @@ public class LauncherOptions { private String queue; - private int defaultParallelism=1; - public String getMode() { return mode; } @@ -162,11 +160,4 @@ public void setQueue(String queue) { this.queue = queue; } - public int getDefaultParallelism() { - return defaultParallelism; - } - - public void setDefaultParallelism(int defaultParallelism) { - this.defaultParallelism = defaultParallelism; - } } diff --git a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java b/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java deleted file mode 100644 index 1879c6ea1..000000000 --- a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Copyright (C) 2018 The Sylph Authors - * - * Licensed 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.yarn; - -import java.util.Objects; -import java.util.Properties; - -public class JobParameter -{ - private int parallelism = 1; - private String queue = "default"; - private int taskManagerMemoryMb = 1024; - private int taskManagerCount = 1; - private int taskManagerSlots = 1; - private int jobManagerMemoryMb = 1024; - - public JobParameter() {} - - public JobParameter(Properties confProperties) { - this.parallelism = confProperties.getProperty("parallelism")==null?parallelism:Integer.parseInt(confProperties.getProperty("parallelism")); - this.queue = confProperties.getProperty("queue")==null?queue:confProperties.getProperty("queue"); - this.taskManagerMemoryMb = confProperties.getProperty("taskManagerMemoryMb")==null?taskManagerMemoryMb:Integer.parseInt(confProperties.getProperty("taskManagerMemoryMb")); - this.taskManagerCount = confProperties.getProperty("taskManagerCount")==null?taskManagerCount:Integer.parseInt(confProperties.getProperty("taskManagerCount")); - this.taskManagerSlots = confProperties.getProperty("taskManagerSlots")==null?taskManagerSlots:Integer.parseInt(confProperties.getProperty("taskManagerSlots")); - this.jobManagerMemoryMb = confProperties.getProperty("jobManagerMemoryMb")==null?jobManagerMemoryMb:Integer.parseInt(confProperties.getProperty("jobManagerMemoryMb")); - } - - public JobParameter(int parallelism, String queue, int taskManagerMemoryMb, int taskManagerCount, int taskManagerSlots, int jobManagerMemoryMb) { - this.parallelism = parallelism; - this.queue = queue; - this.taskManagerMemoryMb = taskManagerMemoryMb; - this.taskManagerCount = taskManagerCount; - this.taskManagerSlots = taskManagerSlots; - this.jobManagerMemoryMb = jobManagerMemoryMb; - } - - public void setQueue(String queue) - { - this.queue = queue; - } - - public void setTaskManagerCount(int taskManagerCount) - { - this.taskManagerCount = taskManagerCount; - } - - public void setTaskManagerMemoryMb(int taskManagerMemoryMb) - { - this.taskManagerMemoryMb = taskManagerMemoryMb; - } - - public void setTaskManagerSlots(int taskManagerSlots) - { - this.taskManagerSlots = taskManagerSlots; - } - - public void setJobManagerMemoryMb(int jobManagerMemoryMb) - { - this.jobManagerMemoryMb = jobManagerMemoryMb; - } - - public void setParallelism(int parallelism) - { - this.parallelism = parallelism; - } - - public int getParallelism() - { - return parallelism; - } - - public String getQueue() - { - return queue; - } - - public int getJobManagerMemoryMb() - { - return jobManagerMemoryMb; - } - - public int getTaskManagerSlots() - { - return taskManagerSlots; - } - - public int getTaskManagerCount() - { - return taskManagerCount; - } - - public int getTaskManagerMemoryMb() - { - return taskManagerMemoryMb; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - JobParameter jobParameter = (JobParameter) o; - return Objects.equals(this.queue, jobParameter.queue) && - Objects.equals(this.taskManagerCount, jobParameter.taskManagerCount) && - Objects.equals(this.taskManagerMemoryMb, jobParameter.taskManagerMemoryMb); - } - - @Override - public int hashCode() - { - return Objects.hash(queue, taskManagerMemoryMb, taskManagerCount); - } -} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java deleted file mode 100644 index 79ecbcc36..000000000 --- a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Copyright (C) 2018 The Sylph Authors - * - * Licensed 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.yarn; - -import org.apache.flink.configuration.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -import java.util.Set; - -public class YarnClusterConfiguration { - /** - * The configuration used by YARN (i.e.,
yarn-site.xml
). - */ - private final YarnConfiguration yarnConf; - - /** - * The home directory of all job where all the temporary files for each jobs are stored. - */ - private final String appRootDir; - - /** - * Thepublic YarnClusterConfiguration(Configuration , YarnConfiguration , String , Path , Set ) { - } location of the Flink jar. - */ - private final Path flinkJar; - - /** - * Additional resources to be localized for both JobManager and TaskManager. - * They will NOT be added into the classpaths. - */ - private final Set resourcesToLocalize; - - /** - * flink conf - */ - private final Configuration flinkConfiguration; - - public YarnClusterConfiguration( - Configuration flinkConf, - YarnConfiguration conf, - String appRootDir, - Path flinkJar, - Set resourcesToLocalize) { - this.flinkConfiguration = flinkConf; - this.yarnConf = conf; - this.appRootDir = appRootDir; - this.flinkJar = flinkJar; - this.resourcesToLocalize = resourcesToLocalize; - } - - - - YarnConfiguration yarnConf() { - return yarnConf; - } - - public String appRootDir() { - return appRootDir; - } - - public Configuration flinkConfiguration() { - return flinkConfiguration; - } - - public Path flinkJar() { - return flinkJar; - } - - public Set resourcesToLocalize() { - return resourcesToLocalize; - } - -} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java deleted file mode 100644 index d442d16f2..000000000 --- a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java +++ /dev/null @@ -1,393 +0,0 @@ -/* - * Copyright (C) 2018 The Sylph Authors - * - * Licensed 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.yarn; - -import org.apache.flink.client.deployment.ClusterDeploymentException; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.yarn.*; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.*; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.util.ConverterUtils; -import org.apache.hadoop.yarn.util.Records; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.util.*; -import java.util.concurrent.*; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.yarn.api.records.YarnApplicationState.NEW; - -public class YarnClusterDescriptor - extends AbstractYarnClusterDescriptor -{ - private static final String APPLICATION_TYPE = "58_FLINK"; - private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); - private static final int MAX_ATTEMPT = 1; - private static final long DEPLOY_TIMEOUT_MS = 600 * 1000; - private static final long RETRY_DELAY_MS = 250; - private static final ScheduledExecutorService YARN_POLL_EXECUTOR = Executors.newSingleThreadScheduledExecutor(); - - private final YarnClusterConfiguration clusterConf; - private final YarnClient yarnClient; - private final JobParameter appConf; - private final Path homedir; - private final ApplicationId yarnAppId; - private final String jobName; - private final Iterable userProvidedJars; - private Path flinkJar; - - public YarnClusterDescriptor( - final YarnClusterConfiguration clusterConf, - final YarnClient yarnClient, - final JobParameter appConf, - ApplicationId yarnAppId, - String jobName, - Iterable userProvidedJars) - { - super(clusterConf.flinkConfiguration(), clusterConf.yarnConf(), clusterConf.appRootDir(), yarnClient, false); - this.jobName = jobName; - this.clusterConf = clusterConf; - this.yarnClient = yarnClient; - this.appConf = appConf; - this.yarnAppId = yarnAppId; - this.userProvidedJars = userProvidedJars; - this.homedir = new Path(clusterConf.appRootDir(), yarnAppId.toString()); - } - - @Override - protected String getYarnSessionClusterEntrypoint() - { - return YarnApplicationMasterRunner.class.getName(); - } - - /** - * 提交到yarn时 任务启动入口类 - */ - @Override - protected String getYarnJobClusterEntrypoint() - { - return YarnApplicationMasterRunner.class.getName(); - } - - @Override - protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) - throws Exception - { - return new RestClusterClient<>( - flinkConfiguration, - report.getApplicationId()); - } - - @Override - public YarnClient getYarnClient() - { - return this.yarnClient; - } - - public YarnClusterClient deploy() - { - ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); - context.setApplicationId(yarnAppId); - try { - ApplicationReport report = startAppMaster(context); - - Configuration conf = getFlinkConfiguration(); - conf.setString(JobManagerOptions.ADDRESS.key(), report.getHost()); - conf.setInteger(JobManagerOptions.PORT.key(), report.getRpcPort()); - - return new YarnClusterClient(this, - appConf.getTaskManagerCount(), - appConf.getTaskManagerSlots(), - report, conf, false); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private ApplicationReport startAppMaster(ApplicationSubmissionContext appContext) - throws Exception - { - ApplicationId appId = appContext.getApplicationId(); - appContext.setMaxAppAttempts(MAX_ATTEMPT); - - Map localResources = new HashMap<>(); - Set shippedPaths = new HashSet<>(); - collectLocalResources(localResources, shippedPaths); - - final ContainerLaunchContext amContainer = setupApplicationMasterContainer( - getYarnJobClusterEntrypoint(), - false, - true, - false, - appConf.getJobManagerMemoryMb() - ); - - amContainer.setLocalResources(localResources); - - final String classPath = String.join(File.pathSeparator, localResources.keySet()); - - final String shippedFiles = shippedPaths.stream() - .map(path -> path.getName() + "=" + path) - .collect(Collectors.joining(",")); - - // Setup CLASSPATH and environment variables for ApplicationMaster - final Map appMasterEnv = setUpAmEnvironment( - appId, - classPath,shippedFiles, - //"","", - getDynamicPropertiesEncoded() - ); - - amContainer.setEnvironment(appMasterEnv); - - // Set up resource type requirements for ApplicationMaster - Resource capability = Records.newRecord(Resource.class); - capability.setMemory(appConf.getJobManagerMemoryMb()); //设置jobManneger - capability.setVirtualCores(1); //默认是1 - - appContext.setApplicationName(jobName); - appContext.setApplicationType(APPLICATION_TYPE); - appContext.setAMContainerSpec(amContainer); - appContext.setResource(capability); - //appContext.setApplicationTags(appConf.getAppTags()); - if (appConf.getQueue() != null) { - appContext.setQueue(appConf.getQueue()); - } - - LOG.info("Submitting application master {}", appId); - yarnClient.submitApplication(appContext); - - PollDeploymentStatus poll = new PollDeploymentStatus(appId); - YARN_POLL_EXECUTOR.submit(poll); - try { - return poll.result.get(); - } - catch (ExecutionException e) { - LOG.warn("Failed to deploy {}, cause: {}", appId.toString(), e.getCause()); - yarnClient.killApplication(appId); - throw (Exception) e.getCause(); - } - } - - private void collectLocalResources( - Map resources, - Set shippedPaths - ) - throws IOException, URISyntaxException - { - if(clusterConf.flinkJar() != null) { - Path flinkJar = clusterConf.flinkJar(); - LocalResource flinkJarResource = setupLocalResource(flinkJar, homedir, ""); //放到 Appid/根目录下 - this.flinkJar = ConverterUtils.getPathFromYarnURL(flinkJarResource.getResource()); - resources.put("flink.jar", flinkJarResource); - } - if(clusterConf.resourcesToLocalize() != null) { - for (Path p : clusterConf.resourcesToLocalize()) { //主要是 flink.jar log4f.propors 和 flink.yaml 三个文件 - LocalResource resource = setupLocalResource(p, homedir, ""); //这些需要放到根目录下 - resources.put(p.getName(), resource); - if ("log4j.properties".equals(p.getName())) { - shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); - } - } - } - if(userProvidedJars != null) { - for (Path p : userProvidedJars) { - String name = p.getName(); - if (resources.containsKey(name)) { //这里当jar 有重复的时候 会抛出异常 - LOG.warn("Duplicated name in the shipped files {}", p); - } else { - LocalResource resource = setupLocalResource(p, homedir, "jars"); //这些放到 jars目录下 - resources.put(name, resource); - shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); - } - } - } - } - - private LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) - throws IOException - { - LocalResource localResource = Records.newRecord(LocalResource.class); - FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); - localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); - localResource.setSize(jarStat.getLen()); - localResource.setTimestamp(jarStat.getModificationTime()); - localResource.setType(LocalResourceType.FILE); - localResource.setVisibility(LocalResourceVisibility.APPLICATION); - return localResource; - } - - private LocalResource setupLocalResource( - Path localSrcPath, - Path homedir, - String relativeTargetPath) - throws IOException - { - if (new File(localSrcPath.toUri().getPath()).isDirectory()) { - throw new IllegalArgumentException("File to copy must not be a directory: " + - localSrcPath); - } - - // copy resource to HDFS - String suffix = "." + (relativeTargetPath.isEmpty() ? "" : "/" + relativeTargetPath) - + "/" + localSrcPath.getName(); - - Path dst = new Path(homedir, suffix); - - LOG.info("Uploading {}", dst); - - FileSystem hdfs = FileSystem.get(clusterConf.yarnConf()); - hdfs.copyFromLocalFile(false, true, localSrcPath, dst); - - // now create the resource instance - LocalResource resource = registerLocalResource(hdfs, dst); - return resource; - } - - private Map setUpAmEnvironment( - ApplicationId appId, - String amClassPath, - String shipFiles, - String dynamicProperties) - throws IOException, URISyntaxException - { - final Map appMasterEnv = new HashMap<>(); - - // set Flink app class path - appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, amClassPath); - - // set Flink on YARN internal configuration values - appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(appConf.getTaskManagerCount())); - appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(appConf.getTaskManagerMemoryMb())); - appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(appConf.getTaskManagerSlots())); - appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, flinkJar.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homedir.toString()); //$home/.flink/appid 这个目录里面存放临时数据 - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, shipFiles); - - appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(true)); //是否分离 分离就cluser模式 否则是client模式 - - appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, - UserGroupInformation.getCurrentUser().getUserName()); - - if (dynamicProperties != null) { - appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicProperties); - } - - // set classpath from YARN configuration - Utils.setupYarnClassPath(clusterConf.yarnConf(), appMasterEnv); - - return appMasterEnv; - } - - /** - * flink 1.5 add - */ - @Override - public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) - throws ClusterDeploymentException - { - throw new UnsupportedOperationException("this method have't support!"); - } - - private final class PollDeploymentStatus - implements Runnable - { - private final CompletableFuture result = new CompletableFuture<>(); - private final ApplicationId appId; - private YarnApplicationState lastAppState = NEW; - private long startTime; - - private PollDeploymentStatus(ApplicationId appId) - { - this.appId = appId; - } - - @Override - public void run() - { - if (startTime == 0) { - startTime = System.currentTimeMillis(); - } - - try { - ApplicationReport report = poll(); - if (report == null) { - YARN_POLL_EXECUTOR.schedule(this, RETRY_DELAY_MS, TimeUnit.MILLISECONDS); - } - else { - result.complete(report); - } - } - catch (YarnException | IOException e) { - result.completeExceptionally(e); - } - } - - private ApplicationReport poll() - throws IOException, YarnException - { - ApplicationReport report; - report = yarnClient.getApplicationReport(appId); - YarnApplicationState appState = report.getYarnApplicationState(); - LOG.debug("Application State: {}", appState); - - switch (appState) { - case FAILED: - case FINISHED: - //TODO: the finished state may be valid in flip-6 - case KILLED: - throw new IOException("The YARN application unexpectedly switched to state " - + appState + " during deployment. \n" - + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" - + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" - + "yarn logs -applicationId " + appId); - //break .. - case RUNNING: - LOG.info("YARN application has been deployed successfully."); - break; - default: - if (appState != lastAppState) { - LOG.info("Deploying cluster, current state " + appState); - } - lastAppState = appState; - if (System.currentTimeMillis() - startTime > DEPLOY_TIMEOUT_MS) { - throw new RuntimeException(String.format("Deployment took more than %d seconds. " - + "Please check if the requested resources are available in the YARN cluster", DEPLOY_TIMEOUT_MS)); - } - return null; - } - return report; - } - } -} diff --git a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala deleted file mode 100644 index e120addb3..000000000 --- a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala +++ /dev/null @@ -1,482 +0,0 @@ -package org.apache.flink.table.calcite - -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.core._ -import org.apache.calcite.rel.logical._ -import org.apache.calcite.rel.{RelNode, RelShuttle} -import org.apache.calcite.rex._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo -import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} -import org.apache.flink.table.functions.sql.ProctimeSqlFunction -import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate -import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType -import org.apache.flink.table.validate.BasicOperatorTable - -import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ -import scala.collection.mutable - -/** - * Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a - * time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in - * some cases, but not all. - */ -class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { - - private val timestamp = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) - - override def visit(intersect: LogicalIntersect): RelNode = - throw new TableException("Logical intersect in a stream environment is not supported yet.") - - override def visit(union: LogicalUnion): RelNode = { - // visit children and update inputs - val inputs = union.getInputs.map(_.accept(this)) - - // make sure that time indicator types match - val inputTypes = inputs.map(_.getRowType) - - val head = inputTypes.head.getFieldList.map(_.getType) - - val isValid = inputTypes.forall { t => - val fieldTypes = t.getFieldList.map(_.getType) - - fieldTypes.zip(head).forall { case (l, r) => - // check if time indicators match - if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { - val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime - val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime - leftTime == rightTime - } - // one side is not an indicator - else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { - false - } - // uninteresting types - else { - true - } - } - } - - if (!isValid) { - throw new ValidationException( - "Union fields with time attributes have different types.") - } - - LogicalUnion.create(inputs, union.all) - } - - override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate) - - override def visit(minus: LogicalMinus): RelNode = - throw new TableException("Logical minus in a stream environment is not supported yet.") - - override def visit(sort: LogicalSort): RelNode = { - - val input = sort.getInput.accept(this) - LogicalSort.create(input, sort.collation, sort.offset, sort.fetch) - } - - // override def visit(`match`: LogicalMatch): RelNode = - // throw new TableException("Logical match in a stream environment is not supported yet.") - // and cpe entry - override def visit(`match`: LogicalMatch): RelNode = { - val rowType = `match`.getInput.getRowType - - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - rowType.getFieldList.map(_.getType)) - - val patternDefinitions = - `match`.getPatternDefinitions.foldLeft(mutable.Map[String, RexNode]()) { - case (m, (k, v)) => m += k -> v.accept(materializer) - } - - val measures = `match`.getMeasures.foldLeft(mutable.Map[String, RexNode]()) { - case (m, (k, v)) => m += k -> v.accept(materializer) - } - - val outputTypeBuilder = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .builder() - - `match`.getRowType.getFieldList.asScala - .foreach(x => measures.get(x.getName) match { - case Some(measure) => outputTypeBuilder.add(x.getName, measure.getType) - case None => outputTypeBuilder.add(x) - }) - - LogicalMatch.create( - `match`.getInput, - outputTypeBuilder.build(), - `match`.getPattern, - `match`.isStrictStart, - `match`.isStrictEnd, - patternDefinitions, - measures, - `match`.getAfter, - `match`.getSubsets.asInstanceOf[java.util.Map[String, java.util.TreeSet[String]]], - `match`.isAllRows, - `match`.getPartitionKeys, - `match`.getOrderKeys, - `match`.getInterval) - } - - - override def visit(other: RelNode): RelNode = other match { - - case uncollect: Uncollect => - // visit children and update inputs - val input = uncollect.getInput.accept(this) - Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality) - - case scan: LogicalTableFunctionScan => - scan - - case aggregate: LogicalWindowAggregate => - val convAggregate = convertAggregate(aggregate) - - LogicalWindowAggregate.create( - aggregate.getWindow, - aggregate.getNamedProperties, - convAggregate) - - case _ => - throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") - } - - - override def visit(exchange: LogicalExchange): RelNode = - throw new TableException("Logical exchange in a stream environment is not supported yet.") - - override def visit(scan: TableScan): RelNode = scan - - override def visit(scan: TableFunctionScan): RelNode = - throw new TableException("Table function scan in a stream environment is not supported yet.") - - override def visit(values: LogicalValues): RelNode = values - - override def visit(filter: LogicalFilter): RelNode = { - // visit children and update inputs - val input = filter.getInput.accept(this) - - // We do not materialize time indicators in conditions because they can be locally evaluated. - // Some conditions are evaluated by special operators (e.g., time window joins). - // Time indicators in remaining conditions are materialized by Calc before the code generation. - LogicalFilter.create(input, filter.getCondition) - } - - override def visit(project: LogicalProject): RelNode = { - // visit children and update inputs - val input = project.getInput.accept(this) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - input.getRowType.getFieldList.map(_.getType)) - - val projects = project.getProjects.map(_.accept(materializer)) - val fieldNames = project.getRowType.getFieldNames - LogicalProject.create(input, projects, fieldNames) - } - - override def visit(join: LogicalJoin): RelNode = { - val left = join.getLeft.accept(this) - val right = join.getRight.accept(this) - - LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType) - - } - - - override def visit(correlate: LogicalCorrelate): RelNode = { - // visit children and update inputs - val inputs = correlate.getInputs.map(_.accept(this)) - - val right = inputs(1) match { - case scan: LogicalTableFunctionScan => - // visit children and update inputs - val scanInputs = scan.getInputs.map(_.accept(this)) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - inputs.head.getRowType.getFieldList.map(_.getType)) - - val call = scan.getCall.accept(materializer) - LogicalTableFunctionScan.create( - scan.getCluster, - scanInputs, - call, - scan.getElementType, - scan.getRowType, - scan.getColumnMappings) - - case _ => - inputs(1) - } - - LogicalCorrelate.create( - inputs.head, - right, - correlate.getCorrelationId, - correlate.getRequiredColumns, - correlate.getJoinType) - } - - - - - private def convertAggregate(aggregate: Aggregate): LogicalAggregate = { - // visit children and update inputs - val input = aggregate.getInput.accept(this) - - // add a project to materialize aggregation arguments/grouping keys - - val refIndices = mutable.Set[Int]() - - // check arguments of agg calls - aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) { - // count(*) has an empty argument list - (0 until input.getRowType.getFieldCount).foreach(refIndices.add) - } else { - // for other aggregations - call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add) - }) - - // check grouping sets - aggregate.getGroupSets.foreach(set => - set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add) - ) - - val needsMaterialization = refIndices.exists(idx => - isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType)) - - // create project if necessary - val projectedInput = if (needsMaterialization) { - - // insert or merge with input project if - // a time attribute is accessed and needs to be materialized - input match { - - // merge - case lp: LogicalProject => - val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) => - if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) { - if (isRowtimeIndicatorType(expr.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast(timestamp, expr) - } else { - // generate proctime access - rexBuilder.makeCall(ProctimeSqlFunction, expr) - } - } else { - expr - } - } - - LogicalProject.create( - lp.getInput, - projects, - input.getRowType.getFieldNames) - - // new project - case _ => - val projects = input.getRowType.getFieldList.map { field => - if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) { - if (isRowtimeIndicatorType(field.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast( - timestamp, - new RexInputRef(field.getIndex, field.getType)) - } else { - // generate proctime access - rexBuilder.makeCall( - ProctimeSqlFunction, - new RexInputRef(field.getIndex, field.getType)) - } - } else { - new RexInputRef(field.getIndex, field.getType) - } - } - - LogicalProject.create( - input, - projects, - input.getRowType.getFieldNames) - } - } else { - // no project necessary - input - } - - // remove time indicator type as agg call return type - val updatedAggCalls = aggregate.getAggCallList.map { call => - val callType = if (isTimeIndicatorType(call.getType)) { - timestamp - } else { - call.getType - } - AggregateCall.create( - call.getAggregation, - call.isDistinct, - call.getArgList, - call.filterArg, - callType, - call.name) - } - - LogicalAggregate.create( - projectedInput, - aggregate.indicator, - aggregate.getGroupSet, - aggregate.getGroupSets, - updatedAggCalls) - } - -} - -object RelTimeIndicatorConverter { - - def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = { - val converter = new RelTimeIndicatorConverter(rexBuilder) - val convertedRoot = rootRel.accept(converter) - - var needsConversion = false - - // materialize remaining proctime indicators - val projects = convertedRoot.getRowType.getFieldList.map(field => - if (isProctimeIndicatorType(field.getType)) { - needsConversion = true - rexBuilder.makeCall( - ProctimeSqlFunction, - new RexInputRef(field.getIndex, field.getType)) - } else { - new RexInputRef(field.getIndex, field.getType) - } - ) - - // add final conversion if necessary - if (needsConversion) { - LogicalProject.create( - convertedRoot, - projects, - convertedRoot.getRowType.getFieldNames) - } else { - convertedRoot - } - } - - /** - * Materializes time indicator accesses in an expression. - * - * @param expr The expression in which time indicators are materialized. - * @param rowType The input schema of the expression. - * @param rexBuilder A RexBuilder. - * - * @return The expression with materialized time indicators. - */ - def convertExpression(expr: RexNode, rowType: RelDataType, rexBuilder: RexBuilder): RexNode = { - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - rowType.getFieldList.map(_.getType)) - - expr.accept(materializer) - } -} - -class RexTimeIndicatorMaterializer( - private val rexBuilder: RexBuilder, - private val input: Seq[RelDataType]) - extends RexShuttle { - - private val timestamp = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) - - override def visitInputRef(inputRef: RexInputRef): RexNode = { - // reference is interesting - if (isTimeIndicatorType(inputRef.getType)) { - val resolvedRefType = input(inputRef.getIndex) - // input is a valid time indicator - if (isTimeIndicatorType(resolvedRefType)) { - inputRef - } - // input has been materialized - else { - new RexInputRef(inputRef.getIndex, resolvedRefType) - } - } - // reference is a regular field - else { - super.visitInputRef(inputRef) - } - } - - override def visitCall(call: RexCall): RexNode = { - val updatedCall = super.visitCall(call).asInstanceOf[RexCall] - - // materialize operands with time indicators - val materializedOperands = updatedCall.getOperator match { - - // skip materialization for special operators - case BasicOperatorTable.SESSION | BasicOperatorTable.HOP | BasicOperatorTable.TUMBLE => - updatedCall.getOperands.toList - - case _ => - updatedCall.getOperands.map { o => - if (isTimeIndicatorType(o.getType)) { - if (isRowtimeIndicatorType(o.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast(timestamp, o) - } else { - // generate proctime access - rexBuilder.makeCall(ProctimeSqlFunction, o) - } - } else { - o - } - } - } - - // remove time indicator return type - updatedCall.getOperator match { - - // we do not modify AS if operand has not been materialized - case SqlStdOperatorTable.AS if - isTimeIndicatorType(updatedCall.getOperands.get(0).getType) => - updatedCall - - // do not modify window time attributes - case BasicOperatorTable.TUMBLE_ROWTIME | - BasicOperatorTable.TUMBLE_PROCTIME | - BasicOperatorTable.HOP_ROWTIME | - BasicOperatorTable.HOP_PROCTIME | - BasicOperatorTable.SESSION_ROWTIME | - BasicOperatorTable.SESSION_PROCTIME - // since we materialize groupings on time indicators, - // we cannot check the operands anymore but the return type at least - if isTimeIndicatorType(updatedCall.getType) => - updatedCall - - // materialize function's result and operands - case _ if isTimeIndicatorType(updatedCall.getType) => - updatedCall.clone(timestamp, materializedOperands) - - // materialize function's operands only - case _ => - updatedCall.clone(updatedCall.getType, materializedOperands) - } - } -} diff --git a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala deleted file mode 100644 index 439e3a7f2..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ /dev/null @@ -1,1920 +0,0 @@ -package org.apache.flink.table.codegen - -import java.math.{BigDecimal => JBigDecimal} - -import org.apache.calcite.avatica.util.DateTimeUtils -import org.apache.calcite.rex._ -import org.apache.calcite.sql.SqlOperator -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{ROW, _} -import org.apache.commons.lang3.StringEscapeUtils -import org.apache.flink.api.common.functions._ -import org.apache.flink.api.common.typeinfo._ -import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.api.java.typeutils._ -import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo -import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.table.api.{TableConfig, TableException} -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.codegen.CodeGenUtils._ -import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE} -import org.apache.flink.table.codegen.calls.ScalarOperators._ -import org.apache.flink.table.codegen.calls.{CurrentTimePointCallGen, FunctionGenerator} -import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions, StreamRecordTimestampSqlFunction} -import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo -import org.apache.flink.table.typeutils.TypeCheckUtils._ -import org.joda.time.format.DateTimeFormatter - -import scala.collection.JavaConversions._ -import scala.collection.mutable - -/** - * [[CodeGenerator]] is the base code generator for generating Flink - * [[org.apache.flink.api.common.functions.Function]]s. - * It is responsible for expression generation and tracks the context (member variables etc). - * - * @param config configuration that determines runtime behavior - * @param nullableInput input(s) can be null. - * @param input1 type information about the first input of the Function - * @param input2 type information about the second input if the Function is binary - * @param input1FieldMapping additional mapping information for input1. - * POJO types have no deterministic field order and some input fields might not be read. - * The input1FieldMapping is also used to inject time indicator attributes. - * @param input2FieldMapping additional mapping information for input2. - * POJO types have no deterministic field order and some input fields might not be read. - */ -abstract class CodeGenerator( - config: TableConfig, - nullableInput: Boolean, - input1: TypeInformation[_ <: Any], - input2: Option[TypeInformation[_ <: Any]] = None, - input1FieldMapping: Option[Array[Int]] = None, - input2FieldMapping: Option[Array[Int]] = None) - extends RexVisitor[GeneratedExpression] { - - // check if nullCheck is enabled when inputs can be null - if (nullableInput && !config.getNullCheck) { - throw new CodeGenException("Null check must be enabled if entire rows can be null.") - } - - // check for POJO input1 mapping - input1 match { - case pt: PojoTypeInfo[_] => - input1FieldMapping.getOrElse( - throw new CodeGenException("No input mapping is specified for input1 of type POJO.")) - case _ => // ok - } - - // check for POJO input2 mapping - input2 match { - case Some(pt: PojoTypeInfo[_]) => - input2FieldMapping.getOrElse( - throw new CodeGenException("No input mapping is specified for input2 of type POJO.")) - case _ => // ok - } - - protected val input1Mapping: Array[Int] = input1FieldMapping match { - case Some(mapping) => mapping - case _ => (0 until input1.getArity).toArray - } - - protected val input2Mapping: Array[Int] = input2FieldMapping match { - case Some(mapping) => mapping - case _ => input2 match { - case Some(input) => (0 until input.getArity).toArray - case _ => Array[Int]() - } - } - - // set of member statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableMemberStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of constructor statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableInitStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of open statements for RichFunction that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableOpenStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of close statements for RichFunction that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableCloseStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of statements that will be added only once per record; - // code should only update member variables because local variables are not accessible if - // the code needs to be split; - // we use a LinkedHashSet to keep the insertion order - protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // map of initial input unboxing expressions that will be added only once - // (inputTerm, index) -> expr - protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = - mutable.Map[(String, Int), GeneratedExpression]() - - // set of constructor statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = - mutable.LinkedHashSet[(String, String)]() - - /** - * Flag that indicates that the generated code needed to be split into several methods. - */ - protected var hasCodeSplits: Boolean = false - - /** - * @return code block of statements that need to be placed in the member area of the Function - * (e.g. member variables and their initialization) - */ - def reuseMemberCode(): String = { - reusableMemberStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the constructor of the Function - */ - def reuseInitCode(): String = { - reusableInitStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the open() method of RichFunction - */ - def reuseOpenCode(): String = { - reusableOpenStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the close() method of RichFunction - */ - def reuseCloseCode(): String = { - reusableCloseStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the SAM of the Function - */ - def reusePerRecordCode(): String = { - reusablePerRecordStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that unbox input variables to a primitive variable - * and a corresponding null flag variable - */ - def reuseInputUnboxingCode(): String = { - reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n") - } - - /** - * @return code block of constructor statements for the Function - */ - def reuseConstructorCode(className: String): String = { - reusableConstructorStatements.map { case (params, body) => - s""" - |public $className($params) throws Exception { - | this(); - | $body - |} - |""".stripMargin - }.mkString("", "\n", "\n") - } - - /** - * @return term of the (casted and possibly boxed) first input - */ - var input1Term = "in1" - - /** - * @return term of the (casted and possibly boxed) second input - */ - var input2Term = "in2" - - /** - * @return term of the (casted) output collector - */ - var collectorTerm = "c" - - /** - * @return term of the output record (possibly defined in the member area e.g. Row, Tuple) - */ - var outRecordTerm = "out" - - /** - * @return term of the [[ProcessFunction]]'s context - */ - var contextTerm = "ctx" - - /** - * @return returns if null checking is enabled - */ - def nullCheck: Boolean = config.getNullCheck - - /** - * Generates an expression from a RexNode. If objects or variables can be reused, they will be - * added to reusable code sections internally. - * - * @param rex Calcite row expression - * @return instance of GeneratedExpression - */ - def generateExpression(rex: RexNode): GeneratedExpression = { - rex.accept(this) - } - - /** - * Generates an expression that converts the first input (and second input) into the given type. - * If two inputs are converted, the second input is appended. If objects or variables can - * be reused, they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param returnType conversion target type. Inputs and output must have the same arity. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @param rowtimeExpression an expression to extract the value of a rowtime field from - * the input data. Required if the field indices include a rowtime - * marker. - * @return instance of GeneratedExpression - */ - def generateConverterResultExpression( - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String], - rowtimeExpression: Option[RexNode] = None) - : GeneratedExpression = { - - val input1AccessExprs = input1Mapping.map { - case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | - TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER if rowtimeExpression.isDefined => - // generate rowtime attribute from expression - generateExpression(rowtimeExpression.get) - case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | - TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER => - throw TableException("Rowtime extraction expression missing. Please report a bug.") - case TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER => - // attribute is proctime indicator. - // we use a null literal and generate a timestamp when we need it. - generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) - case TimeIndicatorTypeInfo.PROCTIME_BATCH_MARKER => - // attribute is proctime field in a batch query. - // it is initialized with the current time. - generateCurrentTimestamp() - case idx => - generateInputAccess(input1, input1Term, idx) - } - - val input2AccessExprs = input2 match { - case Some(ti) => - input2Mapping.map(idx => generateInputAccess(ti, input2Term, idx)).toSeq - case None => Seq() // add nothing - } - - generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames) - } - - /** - * Generates an expression from the left input and the right table function. - */ - def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = { - val input1AccessExprs = input1Mapping.map { idx => - generateInputAccess(input1, input1Term, idx) - } - - val input2AccessExprs = input2 match { - case Some(ti) => - // use generateFieldAccess instead of generateInputAccess to avoid the generated table - // function's field access code is put on the top of function body rather than - // the while loop - input2Mapping.map { idx => - generateFieldAccess(ti, input2Term, idx) - }.toSeq - case None => throw new CodeGenException("Type information of input2 must not be null.") - } - (input1AccessExprs, input2AccessExprs) - } - - /** - * Generates an expression from a sequence of RexNode. If objects or variables can be reused, - * they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param returnType conversion target type. Type must have the same arity than rexNodes. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @param rexNodes sequence of RexNode to be converted - * @return instance of GeneratedExpression - */ - def generateResultExpression( - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String], - rexNodes: Seq[RexNode]) - : GeneratedExpression = { - val fieldExprs = rexNodes.map(generateExpression) - generateResultExpression(fieldExprs, returnType, resultFieldNames) - } - - /** - * Generates an expression from a sequence of other expressions. If objects or variables can - * be reused, they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param fieldExprs field expressions to be converted - * @param returnType conversion target type. Type must have the same arity than fieldExprs. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @return instance of GeneratedExpression - */ - def generateResultExpression( - fieldExprs: Seq[GeneratedExpression], - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String]) - : GeneratedExpression = { - // initial type check - if (returnType.getArity != fieldExprs.length) { - throw new CodeGenException( - s"Arity [${returnType.getArity}] of result type [$returnType] does not match " + - s"number [${fieldExprs.length}] of expressions [$fieldExprs].") - } - if (resultFieldNames.length != fieldExprs.length) { - throw new CodeGenException( - s"Arity [${resultFieldNames.length}] of result field names [$resultFieldNames] does not " + - s"match number [${fieldExprs.length}] of expressions [$fieldExprs].") - } - // type check - returnType match { - case pt: PojoTypeInfo[_] => - fieldExprs.zipWithIndex foreach { - case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression [$fieldExpr] type is" + - s" [${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") - - case _ => // ok - } - - case ct: CompositeType[_] => - fieldExprs.zipWithIndex foreach { - case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression[$fieldExpr] type is " + - s"[${fieldExpr.resultType}], result type is [${ct.getTypeAt(i)}]") - case _ => // ok - } - - case t: TypeInformation[_] if t != fieldExprs.head.resultType => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression [${fieldExprs.head}] " + - s"type is [${fieldExprs.head.resultType}], result type is [$t]") - - case _ => // ok - } - - val returnTypeTerm = boxedTypeTermForTypeInfo(returnType) - val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing) - - // generate result expression - returnType match { - case ri: RowTypeInfo => - addReusableOutRecord(ri) - val resultSetters = boxedFieldExprs.zipWithIndex map { - case (fieldExpr, i) => - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | $outRecordTerm.setField($i, null); - |} - |else { - | $outRecordTerm.setField($i, ${fieldExpr.resultTerm}); - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.setField($i, ${fieldExpr.resultTerm}); - |""".stripMargin - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case pt: PojoTypeInfo[_] => - addReusableOutRecord(pt) - val resultSetters = boxedFieldExprs.zip(resultFieldNames) map { - case (fieldExpr, fieldName) => - val accessor = getFieldAccessor(pt.getTypeClass, fieldName) - - accessor match { - // Reflective access of primitives/Objects - case ObjectPrivateFieldAccessor(field) => - val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName) - - val defaultIfNull = if (isFieldPrimitive(field)) { - primitiveDefaultValue(fieldExpr.resultType) - } else { - "null" - } - - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | ${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - defaultIfNull)}; - |} - |else { - | ${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - fieldExpr.resultTerm)}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - fieldExpr.resultTerm)}; - |""".stripMargin - } - - // primitive or Object field access (implicit boxing) - case _ => - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | $outRecordTerm.$fieldName = null; - |} - |else { - | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |""".stripMargin - } - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case tup: TupleTypeInfo[_] => - addReusableOutRecord(tup) - val resultSetters = boxedFieldExprs.zipWithIndex map { - case (fieldExpr, i) => - val fieldName = "f" + i - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be stored in a Tuple."); - |} - |else { - | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |""".stripMargin - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case _: CaseClassTypeInfo[_] => - val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n") - val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ") - val resultTerm = newName(outRecordTerm) - - val nullCheckCode = if (nullCheck) { - boxedFieldExprs map { (fieldExpr) => - s""" - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be stored in a Case Class."); - |} - |""".stripMargin - } mkString "\n" - } else { - "" - } - - val resultCode = - s""" - |$fieldCodes - |$nullCheckCode - |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams); - |""".stripMargin - - // case classes are not splittable - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, returnType) - - case _: TypeInformation[_] => - val fieldExpr = boxedFieldExprs.head - val nullCheckCode = if (nullCheck) { - s""" - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be used for atomic types."); - |} - |""".stripMargin - } else { - "" - } - val resultCode = - s""" - |${fieldExpr.code} - |$nullCheckCode - |""".stripMargin - - // other types are not splittable - GeneratedExpression(fieldExpr.resultTerm, fieldExpr.nullTerm, resultCode, returnType) - - case _ => - throw new CodeGenException(s"Unsupported result type: $returnType") - } - } - - // ---------------------------------------------------------------------------------------------- - // RexVisitor methods - // ---------------------------------------------------------------------------------------------- - - override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = { - // if inputRef index is within size of input1 we work with input1, input2 otherwise - val input = if (inputRef.getIndex < input1.getArity) { - (input1, input1Term) - } else { - (input2.getOrElse(throw new CodeGenException("Invalid input access.")), input2Term) - } - - val index = if (input._2 == input1Term) { - inputRef.getIndex - } else { - inputRef.getIndex - input1.getArity - } - - generateInputAccess(input._1, input._2, index) - } - - override def visitTableInputRef(rexTableInputRef: RexTableInputRef): GeneratedExpression = - visitInputRef(rexTableInputRef) - - override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = { - val refExpr = rexFieldAccess.getReferenceExpr.accept(this) - val index = rexFieldAccess.getField.getIndex - generateFieldAccess(refExpr, index) - } - - override def visitLiteral(literal: RexLiteral): GeneratedExpression = { - val resultType = FlinkTypeFactory.toTypeInfo(literal.getType) - val value = literal.getValue3 - // null value with type - if (value == null) { - return generateNullLiteral(resultType) - } - // non-null values - literal.getType.getSqlTypeName match { - - case BOOLEAN => - generateNonNullLiteral(resultType, literal.getValue3.toString) - - case TINYINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidByte) { - generateNonNullLiteral(resultType, decimal.byteValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to byte.") - } - - case SMALLINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidShort) { - generateNonNullLiteral(resultType, decimal.shortValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to short.") - } - - case INTEGER => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(resultType, decimal.intValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to integer.") - } - - case BIGINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral(resultType, decimal.longValue().toString + "L") - } - else { - throw new CodeGenException("Decimal can not be converted to long.") - } - - case FLOAT => - val floatValue = value.asInstanceOf[JBigDecimal].floatValue() - floatValue match { - case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN") - case Float.NegativeInfinity => - generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY") - case Float.PositiveInfinity => - generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY") - case _ => generateNonNullLiteral(resultType, floatValue.toString + "f") - } - - case DOUBLE => - val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue() - doubleValue match { - case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN") - case Double.NegativeInfinity => - generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY") - case Double.PositiveInfinity => - generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY") - case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d") - } - case DECIMAL => - val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(resultType, decimalField) - - case VARCHAR | CHAR => - val escapedValue = StringEscapeUtils.escapeJava( - StringEscapeUtils.unescapeJava(value.toString) - ) - generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") - - case SYMBOL => - generateSymbol(value.asInstanceOf[Enum[_]]) - - case DATE => - generateNonNullLiteral(resultType, value.toString) - - case TIME => - generateNonNullLiteral(resultType, value.toString) - - case TIMESTAMP => - generateNonNullLiteral(resultType, value.toString + "L") - - case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(resultType, decimal.intValue().toString) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of months.") - } - - case typeName if DAY_INTERVAL_TYPES.contains(typeName) => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral(resultType, decimal.longValue().toString + "L") - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of milliseconds.") - } - - case t@_ => - throw new CodeGenException(s"Type not supported: $t") - } - } - - override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { - GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1) - } - - override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression = - throw new CodeGenException("Local variables are not supported yet.") - - override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression = - throw new CodeGenException("Range references are not supported yet.") - - override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression = - throw new CodeGenException("Dynamic parameter references are not supported yet.") - - override def visitCall(call: RexCall): GeneratedExpression = { - - // special case: time materialization - if (call.getOperator == ProctimeSqlFunction) { - return generateProctimeTimestamp() - } - - val resultType = FlinkTypeFactory.toTypeInfo(call.getType) - - // convert operands and help giving untyped NULL literals a type - val operands = call.getOperands.zipWithIndex.map { - - // this helps e.g. for AS(null) - // we might need to extend this logic in case some rules do not create typed NULLs - case (operandLiteral: RexLiteral, 0) if - operandLiteral.getType.getSqlTypeName == SqlTypeName.NULL && - call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => - generateNullLiteral(resultType) - - case (o@_, _) => - o.accept(this) - } - generateCall(call, operands, resultType) - } - def generateCall( - call: RexCall, - operands: Seq[GeneratedExpression], - resultType: TypeInformation[_]) - : GeneratedExpression = { - - call.getOperator match { - // arithmetic - case PLUS if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("+", nullCheck, resultType, left, right, config) - - case PLUS | DATETIME_PLUS if isTemporal(resultType) => - val left = operands.head - val right = operands(1) - requireTemporal(left) - requireTemporal(right) - generateTemporalPlusMinus(plus = true, nullCheck, left, right, config) - - case MINUS if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("-", nullCheck, resultType, left, right, config) - - case MINUS | MINUS_DATE if isTemporal(resultType) => - val left = operands.head - val right = operands(1) - requireTemporal(left) - requireTemporal(right) - generateTemporalPlusMinus(plus = false, nullCheck, left, right, config) - - case MULTIPLY if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right, config) - - case MULTIPLY if isTimeInterval(resultType) => - val left = operands.head - val right = operands(1) - requireTimeInterval(left) - requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right, config) - - case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("/", nullCheck, resultType, left, right, config) - - case MOD if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("%", nullCheck, resultType, left, right, config) - - case UNARY_MINUS if isNumeric(resultType) => - val operand = operands.head - requireNumeric(operand) - generateUnaryArithmeticOperator("-", nullCheck, resultType, operand) - - case UNARY_MINUS if isTimeInterval(resultType) => - val operand = operands.head - requireTimeInterval(operand) - generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand) - - case UNARY_PLUS if isNumeric(resultType) => - val operand = operands.head - requireNumeric(operand) - generateUnaryArithmeticOperator("+", nullCheck, resultType, operand) - - case UNARY_PLUS if isTimeInterval(resultType) => - val operand = operands.head - requireTimeInterval(operand) - generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand) - - // comparison - case EQUALS => - val left = operands.head - val right = operands(1) - generateEquals(nullCheck, left, right) - - case NOT_EQUALS => - val left = operands.head - val right = operands(1) - generateNotEquals(nullCheck, left, right) - - case GREATER_THAN => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison(">", nullCheck, left, right) - - case GREATER_THAN_OR_EQUAL => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison(">=", nullCheck, left, right) - - case LESS_THAN => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison("<", nullCheck, left, right) - - case LESS_THAN_OR_EQUAL => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison("<=", nullCheck, left, right) - - case IS_NULL => - val operand = operands.head - generateIsNull(nullCheck, operand) - - case IS_NOT_NULL => - val operand = operands.head - generateIsNotNull(nullCheck, operand) - - // logic - case AND => - operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => - requireBoolean(left) - requireBoolean(right) - generateAnd(nullCheck, left, right) - } - - case OR => - operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => - requireBoolean(left) - requireBoolean(right) - generateOr(nullCheck, left, right) - } - - case NOT => - val operand = operands.head - requireBoolean(operand) - generateNot(nullCheck, operand) - - case CASE => - generateIfElse(nullCheck, operands, resultType) - - case IS_TRUE => - val operand = operands.head - requireBoolean(operand) - generateIsTrue(operand) - - case IS_NOT_TRUE => - val operand = operands.head - requireBoolean(operand) - generateIsNotTrue(operand) - - case IS_FALSE => - val operand = operands.head - requireBoolean(operand) - generateIsFalse(operand) - - case IS_NOT_FALSE => - val operand = operands.head - requireBoolean(operand) - generateIsNotFalse(operand) - - case IN => - val left = operands.head - val right = operands.tail - generateIn(this, left, right) - - // casting - case CAST | REINTERPRET => - val operand = operands.head - generateCast(nullCheck, operand, resultType) - - // as / renaming - case AS => - operands.head - - // string arithmetic - case CONCAT => - val left = operands.head - val right = operands(1) - requireString(left) - generateArithmeticOperator("+", nullCheck, resultType, left, right, config) - - // rows - case ROW => - generateRow(this, resultType, operands) - - // arrays - case ARRAY_VALUE_CONSTRUCTOR => - generateArray(this, resultType, operands) - - // maps - case MAP_VALUE_CONSTRUCTOR => - generateMap(this, resultType, operands) - - case ITEM => - operands.head.resultType match { - case t: TypeInformation[_] if isArray(t) => - val array = operands.head - val index = operands(1) - requireInteger(index) - generateArrayElementAt(this, array, index) - - case t: TypeInformation[_] if isMap(t) => - val key = operands(1) - generateMapGet(this, operands.head, key) - - case _ => throw new CodeGenException("Expect an array or a map.") - } - - case CARDINALITY => - operands.head.resultType match { - case t: TypeInformation[_] if isArray(t) => - val array = operands.head - generateArrayCardinality(nullCheck, array) - - case t: TypeInformation[_] if isMap(t) => - val map = operands.head - generateMapCardinality(nullCheck, map) - - case _ => throw new CodeGenException("Expect an array or a map.") - } - - case ELEMENT => - val array = operands.head - requireArray(array) - generateArrayElement(this, array) - - case DOT => - // Due to https://issues.apache.org/jira/browse/CALCITE-2162, expression such as - // "array[1].a.b" won't work now. - if (operands.size > 2) { - throw new CodeGenException( - "A DOT operator with more than 2 operands is not supported yet.") - } - val fieldName = call.operands.get(1).asInstanceOf[RexLiteral].getValueAs(classOf[String]) - val fieldIdx = operands - .head - .resultType - .asInstanceOf[CompositeType[_]] - .getFieldIndex(fieldName) - generateFieldAccess(operands.head, fieldIdx) - - case ScalarSqlFunctions.CONCAT => - generateConcat(this.nullCheck, operands) - - case ScalarSqlFunctions.CONCAT_WS => - generateConcatWs(operands) - - case StreamRecordTimestampSqlFunction => - generateStreamRecordRowtimeAccess() - - // advanced scalar functions - case sqlOperator: SqlOperator => - val callGen = FunctionGenerator.getCallGenerator( - sqlOperator, - operands.map(_.resultType), - resultType) - callGen - .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" + - s"If you think this function should be supported, " + - s"you can create an issue and start a discussion for it.")) - .generate(this, operands) - - // unknown or invalid - case call@_ => - throw new CodeGenException(s"Unsupported call: $call") - } - } - - override def visitOver(over: RexOver): GeneratedExpression = - throw new CodeGenException("Aggregate functions over windows are not supported yet.") - - override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression = - throw new CodeGenException("Subqueries are not supported yet.") - - override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): GeneratedExpression = - throw new CodeGenException("Pattern field references are not supported yet.") - - // ---------------------------------------------------------------------------------------------- - // generator helping methods - // ---------------------------------------------------------------------------------------------- - - private def generateCodeSplits(splits: Seq[String]): String = { - val totalLen = splits.map(_.length + 1).sum // 1 for a line break - - // split - if (totalLen > config.getMaxGeneratedCodeLength) { - - hasCodeSplits = true - - // add input unboxing to member area such that all split functions can access it - reusableInputUnboxingExprs.foreach { case (_, expr) => - - // declaration - val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) - if (nullCheck) { - reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") - } - reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") - - // assignment - if (nullCheck) { - reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") - } - reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") - } - - // add split methods to the member area and return the code necessary to call those methods - val methodCalls = splits.map { split => - val methodName = newName(s"split") - - val method = - s""" - |private final void $methodName() throws Exception { - | $split - |} - |""".stripMargin - reusableMemberStatements.add(method) - - // create method call - s"$methodName();" - } - - methodCalls.mkString("\n") - } - // don't split - else { - splits.mkString("\n") - } - } - - def generateFieldAccess(refExpr: GeneratedExpression, index: Int): GeneratedExpression = { - - val fieldAccessExpr = generateFieldAccess( - refExpr.resultType, - refExpr.resultTerm, - index) - - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType) - val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType) - val resultCode = if (nullCheck) { - s""" - |${refExpr.code} - |$resultTypeTerm $resultTerm; - |boolean $nullTerm; - |if (${refExpr.nullTerm}) { - | $resultTerm = $defaultValue; - | $nullTerm = true; - |} - |else { - | ${fieldAccessExpr.code} - | $resultTerm = ${fieldAccessExpr.resultTerm}; - | $nullTerm = ${fieldAccessExpr.nullTerm}; - |} - |""".stripMargin - } else { - s""" - |${refExpr.code} - |${fieldAccessExpr.code} - |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm}; - |""".stripMargin - } - - GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType) - } - - private def generateInputAccess( - inputType: TypeInformation[_ <: Any], - inputTerm: String, - index: Int) - : GeneratedExpression = { - // if input has been used before, we can reuse the code that - // has already been generated - val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match { - // input access and unboxing has already been generated - case Some(expr) => - expr - - // generate input access and unboxing if necessary - case None => - val expr = if (nullableInput) { - generateNullableInputFieldAccess(inputType, inputTerm, index) - } else { - generateFieldAccess(inputType, inputTerm, index) - } - - reusableInputUnboxingExprs((inputTerm, index)) = expr - expr - } - // hide the generated code as it will be executed only once - GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) - } - - private def generateNullableInputFieldAccess( - inputType: TypeInformation[_ <: Any], - inputTerm: String, - index: Int) - : GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - - val fieldType = inputType match { - case ct: CompositeType[_] => ct.getTypeAt(index) - case t: TypeInformation[_] => t - } - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - val defaultValue = primitiveDefaultValue(fieldType) - val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index) - - val inputCheckCode = - s""" - |$resultTypeTerm $resultTerm; - |boolean $nullTerm; - |if ($inputTerm == null) { - | $resultTerm = $defaultValue; - | $nullTerm = true; - |} - |else { - | ${fieldAccessExpr.code} - | $resultTerm = ${fieldAccessExpr.resultTerm}; - | $nullTerm = ${fieldAccessExpr.nullTerm}; - |} - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType) - } - - def generateFieldAccess( - inputType: TypeInformation[_], - inputTerm: String, - index: Int) - : GeneratedExpression = { - inputType match { - - case ct: CompositeType[_] => - val accessor = fieldAccessorFor(ct, index) - val fieldType: TypeInformation[Any] = ct.getTypeAt(index) - val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) - - accessor match { - case ObjectFieldAccessor(field) => - // primitive - if (isFieldPrimitive(field)) { - generateTerm(fieldType, s"$inputTerm.${field.getName}") - } - // Object - else { - generateInputFieldUnboxing( - fieldType, - s"($fieldTypeTerm) $inputTerm.${field.getName}") - } - - case ObjectGenericFieldAccessor(fieldName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" - generateInputFieldUnboxing(fieldType, inputCode) - - case ObjectMethodAccessor(methodName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" - generateInputFieldUnboxing(fieldType, inputCode) - - case ProductAccessor(i) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)" - generateInputFieldUnboxing(fieldType, inputCode) - - case ObjectPrivateFieldAccessor(field) => - val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) - val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) - // primitive - if (isFieldPrimitive(field)) { - generateTerm(fieldType, reflectiveAccessCode) - } - // Object - else { - generateInputFieldUnboxing(fieldType, reflectiveAccessCode) - } - } - - case t: TypeInformation[_] => - val fieldTypeTerm = boxedTypeTermForTypeInfo(t) - val inputCode = s"($fieldTypeTerm) $inputTerm" - generateInputFieldUnboxing(t, inputCode) - } - } - - private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { - val resultTerm = newName("result") - val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - if (nullCheck) { - val wrappedCode = s""" - |$resultTypeTerm $resultTerm = $defaultValue; - |""".stripMargin - - // mark this expression as a constant literal - GeneratedExpression(resultTerm, ALWAYS_NULL, wrappedCode, resultType, literal = true) - } else { - throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.") - } - } - - private[flink] def generateNonNullLiteral( - literalType: TypeInformation[_], - literalCode: String) - : GeneratedExpression = { - - // mark this expression as a constant literal - generateTerm(literalType, literalCode).copy(literal = true) - } - - private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = { - GeneratedExpression( - qualifyEnum(enum), - NEVER_NULL, - NO_CODE, - new GenericTypeInfo(enum.getDeclaringClass)) - } - - /** - * Generates access to a term (e.g. a field) that does not require unboxing logic. - * - * @param fieldType type of field - * @param fieldTerm expression term of field (already unboxed) - * @return internal unboxed field representation - */ - private[flink] def generateTerm( - fieldType: TypeInformation[_], - fieldTerm: String) - : GeneratedExpression = { - val resultTerm = newName("result") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - - val resultCode = s""" - |$resultTypeTerm $resultTerm = $fieldTerm; - |""".stripMargin - - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, fieldType) - } - - /** - * Converts the external boxed format to an internal mostly primitive field representation. - * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External - * objects are converted to their internal representation (Timestamp -> internal timestamp - * in long). - * - * @param fieldType type of field - * @param fieldTerm expression term of field to be unboxed - * @return internal unboxed field representation - */ - private[flink] def generateInputFieldUnboxing( - fieldType: TypeInformation[_], - fieldTerm: String) - : GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - val defaultValue = primitiveDefaultValue(fieldType) - - // explicit unboxing - val unboxedFieldCode = if (isTimePoint(fieldType)) { - timePointToInternalCode(fieldType, fieldTerm) - } else { - fieldTerm - } - - val wrappedCode = if (nullCheck && !isReference(fieldType)) { - // assumes that fieldType is a boxed primitive. - s""" - |boolean $nullTerm = $fieldTerm == null; - |$resultTypeTerm $resultTerm; - |if ($nullTerm) { - | $resultTerm = $defaultValue; - |} - |else { - | $resultTerm = $fieldTerm; - |} - |""".stripMargin - } else if (nullCheck) { - s""" - |boolean $nullTerm = $fieldTerm == null; - |$resultTypeTerm $resultTerm; - |if ($nullTerm) { - | $resultTerm = $defaultValue; - |} - |else { - | $resultTerm = ($resultTypeTerm) $unboxedFieldCode; - |} - |""".stripMargin - } else { - s""" - |$resultTypeTerm $resultTerm = ($resultTypeTerm) $unboxedFieldCode; - |""".stripMargin - } - - GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType) - } - - /** - * Converts the internal mostly primitive field representation to an external boxed format. - * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal - * representations are converted to their external objects (internal timestamp - * in long -> Timestamp). - * - * @param expr expression to be boxed - * @return external boxed field representation - */ - private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = { - expr.resultType match { - // convert internal date/time/timestamp to java.sql.* objects - case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP => - val resultTerm = newName("result") - val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType) - val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm) - - val resultCode = if (nullCheck) { - s""" - |${expr.code} - |$resultTypeTerm $resultTerm; - |if (${expr.nullTerm}) { - | $resultTerm = null; - |} - |else { - | $resultTerm = $convMethod; - |} - |""".stripMargin - } else { - s""" - |${expr.code} - |$resultTypeTerm $resultTerm = $convMethod; - |""".stripMargin - } - - GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType) - - // other types are autoboxed or need no boxing - case _ => expr - } - } - - private[flink] def generateNullableOutputBoxing( - expr: GeneratedExpression, - typeInfo: TypeInformation[_]) - : GeneratedExpression = { - val boxedExpr = generateOutputFieldBoxing(generateCast(nullCheck, expr, typeInfo)) - val boxedTypeTerm = boxedTypeTermForTypeInfo(typeInfo) - val exprOrNull: String = if (nullCheck) { - s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}" - } else { - boxedExpr.resultTerm - } - boxedExpr.copy(resultTerm = exprOrNull) - } - - private[flink] def generateStreamRecordRowtimeAccess(): GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - - val accessCode = - s""" - |Long $resultTerm = $contextTerm.timestamp(); - |if ($resultTerm == null) { - | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " + - | "TimestampAssigner is defined and the stream environment uses the EventTime time " + - | "characteristic."); - |} - |boolean $nullTerm = false; - """.stripMargin - - GeneratedExpression(resultTerm, nullTerm, accessCode, Types.LONG) - } - - private[flink] def generateProctimeTimestamp(): GeneratedExpression = { - val resultTerm = newName("result") - - val resultCode = - s""" - |long $resultTerm = $contextTerm.timerService().currentProcessingTime(); - |""".stripMargin - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) - } - - private[flink] def generateCurrentTimestamp(): GeneratedExpression = { - new CurrentTimePointCallGen(Types.SQL_TIMESTAMP, false).generate(this, Seq()) - } - - // ---------------------------------------------------------------------------------------------- - // Reusable code snippets - // ---------------------------------------------------------------------------------------------- - - /** - * Adds a reusable output record to the member area of the generated [[Function]]. - * The passed [[TypeInformation]] defines the type class to be instantiated. - * - * @param ti type information of type class to be instantiated during runtime - * @return member variable term - */ - def addReusableOutRecord(ti: TypeInformation[_]): Unit = { - val statement = ti match { - case rt: RowTypeInfo => - s""" - |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = - | new ${ti.getTypeClass.getCanonicalName}(${rt.getArity}); - |""".stripMargin - case _ => - s""" - |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = - | new ${ti.getTypeClass.getCanonicalName}(); - |""".stripMargin - } - reusableMemberStatements.add(statement) - } - - /** - * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]]. - * The field can be used for accessing POJO fields more efficiently during runtime, however, - * the field does not have to be public. - * - * @param clazz class of containing field - * @param fieldName name of field to be extracted and instantiated during runtime - * @return member variable term - */ - def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = { - val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName" - val fieldExtraction = - s""" - |final java.lang.reflect.Field $fieldTerm = - | org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField( - | ${clazz.getCanonicalName}.class, "$fieldName"); - |""".stripMargin - reusableMemberStatements.add(fieldExtraction) - - val fieldAccessibility = - s""" - |$fieldTerm.setAccessible(true); - |""".stripMargin - reusableInitStatements.add(fieldAccessibility) - - fieldTerm - } - - /** - * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]]. - * - * @param decimal decimal object to be instantiated during runtime - * @return member variable term - */ - def addReusableDecimal(decimal: JBigDecimal): String = decimal match { - case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO" - case JBigDecimal.ONE => "java.math.BigDecimal.ONE" - case JBigDecimal.TEN => "java.math.BigDecimal.TEN" - case _ => - val fieldTerm = newName("decimal") - val fieldDecimal = - s""" - |final java.math.BigDecimal $fieldTerm = - | new java.math.BigDecimal("${decimal.toString}"); - |""".stripMargin - reusableMemberStatements.add(fieldDecimal) - fieldTerm - } - - /** - * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]]. - * - * The seed parameter must be a literal/constant expression. - * - * @return member variable term - */ - def addReusableRandom(seedExpr: Option[GeneratedExpression]): String = { - val fieldTerm = newName("random") - - val field = - s""" - |final java.util.Random $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = seedExpr match { - case Some(s) if nullCheck => - s""" - |${s.code} - |if(!${s.nullTerm}) { - | $fieldTerm = new java.util.Random(${s.resultTerm}); - |} - |else { - | $fieldTerm = new java.util.Random(); - |} - |""".stripMargin - case Some(s) => - s""" - |${s.code} - |$fieldTerm = new java.util.Random(${s.resultTerm}); - |""".stripMargin - case _ => - s""" - |$fieldTerm = new java.util.Random(); - |""".stripMargin - } - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - /** - * Adds a reusable DateFormatter to the member area of the generated [[Function]]. - * - * @return member variable term - */ - def addReusableDateFormatter(format: GeneratedExpression): String = { - val fieldTerm = newName("dateFormatter") - - val field = - s""" - |final ${classOf[DateTimeFormatter].getCanonicalName} $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = - s""" - |${format.code} - |$fieldTerm = org.apache.flink.table.runtime.functions. - |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm}); - |""".stripMargin - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - /** - * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. - * - * @param function [[UserDefinedFunction]] object to be instantiated during runtime - * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] - * @return member variable term - */ - def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { - val classQualifier = function.getClass.getCanonicalName - val functionSerializedData = UserDefinedFunctionUtils.serialize(function) - val fieldTerm = s"function_${function.functionIdentifier}" - - val fieldFunction = - s""" - |final $classQualifier $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(fieldFunction) - - val functionDeserialization = - s""" - |$fieldTerm = ($classQualifier) - |${UserDefinedFunctionUtils.getClass.getName.stripSuffix("$")} - |.deserialize("$functionSerializedData"); - """.stripMargin - - reusableInitStatements.add(functionDeserialization) - - val openFunction = if (contextTerm != null) { - s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); - """.stripMargin - } else { - s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); - """.stripMargin - } - reusableOpenStatements.add(openFunction) - - val closeFunction = - s""" - |$fieldTerm.close(); - """.stripMargin - reusableCloseStatements.add(closeFunction) - - fieldTerm - } - - /** - * Adds a reusable constructor statement with the given parameter types. - * - * @param parameterTypes The parameter types to construct the function - * @return member variable terms - */ - def addReusableConstructor(parameterTypes: Class[_]*): Array[String] = { - val parameters = mutable.ListBuffer[String]() - val fieldTerms = mutable.ListBuffer[String]() - val body = mutable.ListBuffer[String]() - - parameterTypes.zipWithIndex.foreach { case (t, index) => - val classQualifier = t.getCanonicalName - val fieldTerm = newName(s"instance_${classQualifier.replace('.', '$')}") - val field = s"final $classQualifier $fieldTerm;" - reusableMemberStatements.add(field) - fieldTerms += fieldTerm - parameters += s"$classQualifier arg$index" - body += s"$fieldTerm = arg$index;" - } - - reusableConstructorStatements.add((parameters.mkString(","), body.mkString("", "\n", "\n"))) - - fieldTerms.toArray - } - - /** - * Adds a reusable [[org.apache.flink.types.Row]] - * to the member area of the generated [[Function]]. - */ - def addReusableRow(arity: Int): String = { - val fieldTerm = newName("row") - val fieldRow = - s""" - |final org.apache.flink.types.Row $fieldTerm = - | new org.apache.flink.types.Row($arity); - |""".stripMargin - reusableMemberStatements.add(fieldRow) - fieldTerm - } - - /** - * Adds a reusable array to the member area of the generated [[Function]]. - */ - def addReusableArray(clazz: Class[_], size: Int): String = { - val fieldTerm = newName("array") - val classQualifier = clazz.getCanonicalName // works also for int[] etc. - val initArray = classQualifier.replaceFirst("\\[", s"[$size") - val fieldArray = - s""" - |final $classQualifier $fieldTerm = - | new $initArray; - |""".stripMargin - reusableMemberStatements.add(fieldArray) - fieldTerm - } - - /** - * Adds a reusable hash map to the member area of the generated [[Function]]. - */ - def addReusableMap(): String = { - val fieldTerm = newName("map") - val classQualifier = "java.util.Map" - val initMap = "java.util.HashMap()" - val fieldMap = - s""" - |final $classQualifier $fieldTerm = - | new $initMap; - |""".stripMargin - reusableMemberStatements.add(fieldMap) - fieldTerm - } - - /** - * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableTimestamp(): String = { - val fieldTerm = s"timestamp" - - // declaration - reusableMemberStatements.add(s"private long $fieldTerm;") - - // assignment - val field = - s""" - |$fieldTerm = java.lang.System.currentTimeMillis(); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableLocalTimestamp(): String = { - val fieldTerm = s"localtimestamp" - - val timestamp = addReusableTimestamp() - - // declaration - reusableMemberStatements.add(s"private long $fieldTerm;") - - // assignment - val field = - s""" - |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable time to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableTime(): String = { - val fieldTerm = s"time" - - val timestamp = addReusableTimestamp() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() - val field = - s""" - |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); - |if (time < 0) { - | time += ${DateTimeUtils.MILLIS_PER_DAY}; - |} - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable local time to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableLocalTime(): String = { - val fieldTerm = s"localtime" - - val localtimestamp = addReusableLocalTimestamp() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() - val field = - s""" - |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - - /** - * Adds a reusable date to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableDate(): String = { - val fieldTerm = s"date" - - val timestamp = addReusableTimestamp() - val time = addReusableTime() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() - val field = - s""" - |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); - |if ($time < 0) { - | $fieldTerm -= 1; - |} - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable [[java.util.HashSet]] to the member area of the generated [[Function]]. - * - * @param elements elements to be added to the set (including null) - * @return member variable term - */ - def addReusableSet(elements: Seq[GeneratedExpression]): String = { - val fieldTerm = newName("set") - - val field = - s""" - |final java.util.Set $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val init = - s""" - |$fieldTerm = new java.util.HashSet(); - |""".stripMargin - reusableInitStatements.add(init) - - elements.foreach { element => - val content = - s""" - |${element.code} - |if (${element.nullTerm}) { - | $fieldTerm.add(null); - |} else { - | $fieldTerm.add(${element.resultTerm}); - |} - |""".stripMargin - - reusableInitStatements.add(content) - } - - fieldTerm - } - - /** - * Adds a reusable constant to the member area of the generated [[Function]]. - * - * @param constant constant expression - * @return member variable term - */ - def addReusableBoxedConstant(constant: GeneratedExpression): String = { - require(constant.literal, "Literal expected") - - val fieldTerm = newName("constant") - - val boxed = generateOutputFieldBoxing(constant) - val boxedType = boxedTypeTermForTypeInfo(boxed.resultType) - - val field = - s""" - |final $boxedType $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val init = - s""" - |${boxed.code} - |$fieldTerm = ${boxed.resultTerm}; - |""".stripMargin - reusableInitStatements.add(init) - - fieldTerm - } - - /** - * Adds a reusable MessageDigest to the member area of the generated [[Function]]. - * - * @return member variable term - */ - def addReusableMessageDigest(algorithm: String): String = { - val fieldTerm = newName("messageDigest") - - val field = - s""" - |final java.security.MessageDigest $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = - s""" - |try { - | $fieldTerm = java.security.MessageDigest.getInstance("$algorithm"); - |} catch (java.security.NoSuchAlgorithmException e) { - | throw new RuntimeException("Algorithm for '$algorithm' is not available.", e); - |} - |""".stripMargin - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - def addReusableInitStatement(initStatement: String): Unit = { - reusableInitStatements.add(initStatement) - } - - def addReusableMemberStatement(memberStatement: String): Unit = { - reusableMemberStatements.add(memberStatement) - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala deleted file mode 100644 index d08dd570f..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala +++ /dev/null @@ -1,574 +0,0 @@ -package org.apache.flink.table.codegen - -import java.math.{BigDecimal => JBigDecimal} -import java.util - -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rex._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{CLASSIFIER, FINAL, FIRST, LAST, MATCH_NUMBER, NEXT, PREV, RUNNING} -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.codegen.Indenter.toISC -import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName, primitiveDefaultValue} -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.types.Row - -import scala.collection.JavaConverters._ -import scala.collection.mutable - -/** - * A code generator for generating CEP related functions. - * - * @param config configuration that determines runtime behavior - * @param nullableInput input(s) can be null. - * @param input type information about the first input of the Function - * @param patternNames the names of patterns - * @param generateCondition whether the code generator is generating [[IterativeCondition]] - * @param patternName the name of current pattern - */ -class MatchCodeGenerator( - config: TableConfig, - nullableInput: Boolean, - input: TypeInformation[_ <: Any], - patternNames: Seq[String], - generateCondition: Boolean, - patternName: Option[String] = None) - extends CodeGenerator(config,nullableInput,input){ - /** - * @return term of pattern names - */ - private val patternNameListTerm = newName("patternNameList") - - /** - * @return term of current pattern which is processing - */ - private val currPatternTerm = newName("currPattern") - - /** - * @return term of current event which is processing - */ - private val currEventTerm = newName("currEvent") - - private val buildPatternNameList: String = { - for (patternName <- patternNames) yield - s""" - |$patternNameListTerm.add("$patternName"); - |""".stripMargin - }.mkString("\n") - - def addReusableStatements(): Unit = { - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val memberStatement = - s""" - |$eventTypeTerm $currEventTerm = null; - |String $currPatternTerm = null; - |java.util.List $patternNameListTerm = new java.util.ArrayList(); - |""".stripMargin - addReusableMemberStatement(memberStatement) - - addReusableInitStatement(buildPatternNameList) - } - - /** - * Generates a [[IterativeCondition]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedIterativeCondition - */ - def generateIterativeCondition( - name: String, - bodyCode: String) - : GeneratedIterativeCondition = { - - val funcName = newName(name) - val inputTypeTerm = boxedTypeTermForTypeInfo(input) - - val funcCode = j""" - public class $funcName - extends ${classOf[IterativeCondition[_]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public boolean filter( - Object _in1, ${classOf[IterativeCondition.Context[_]].getCanonicalName} $contextTerm) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedIterativeCondition(funcName, funcCode) - } - - /** - * Generates a [[PatternSelectFunction]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedPatternSelectFunction - */ - def generatePatternSelectFunction( - name: String, - bodyCode: String) - : GeneratedPatternSelectFunction = { - - val funcName = newName(name) - val inputTypeTerm = - classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName - - val funcCode = j""" - public class $funcName - implements ${classOf[PatternSelectFunction[_, _]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public Object select(java.util.Map> _in1) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedPatternSelectFunction(funcName, funcCode) - } - - /** - * Generates a [[PatternFlatSelectFunction]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedPatternFlatSelectFunction - */ - def generatePatternFlatSelectFunction( - name: String, - bodyCode: String) - : GeneratedPatternFlatSelectFunction = { - - val funcName = newName(name) - val inputTypeTerm = - classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName - - val funcCode = j""" - public class $funcName - implements ${classOf[PatternFlatSelectFunction[_, _]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public void flatSelect(java.util.Map> _in1, - org.apache.flink.util.Collector $collectorTerm) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedPatternFlatSelectFunction(funcName, funcCode) - } - - def generateSelectOutputExpression( - partitionKeys: util.List[RexNode], - measures: util.Map[String, RexNode], - returnType: RowSchema) - : GeneratedExpression = { - - val eventNameTerm = newName("event") - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - - // For "ONE ROW PER MATCH", the output columns include: - // 1) the partition columns; - // 2) the columns defined in the measures clause. - val resultExprs = - partitionKeys.asScala.map { case inputRef: RexInputRef => - generateFieldAccess(input, eventNameTerm, inputRef.getIndex) - } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => - generateExpression(measures.get(fieldName)) - } - - val resultExpression = generateResultExpression( - resultExprs, - returnType.typeInfo, - returnType.fieldNames) - - val resultCode = - s""" - |$eventTypeTerm $eventNameTerm = null; - |if (${partitionKeys.size()} > 0) { - | for (java.util.Map.Entry entry : $input1Term.entrySet()) { - | java.util.List value = (java.util.List) entry.getValue(); - | if (value != null && value.size() > 0) { - | $eventNameTerm = ($eventTypeTerm) value.get(0); - | break; - | } - | } - |} - | - |${resultExpression.code} - |""".stripMargin - - resultExpression.copy(code = resultCode) - } - - def generateFlatSelectOutputExpression( - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - measures: util.Map[String, RexNode], - returnType: RowSchema) - : GeneratedExpression = { - - val patternNameTerm = newName("patternName") - val eventNameTerm = newName("event") - val eventNameListTerm = newName("eventList") - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val listTypeTerm = classOf[java.util.List[_]].getCanonicalName - - // For "ALL ROWS PER MATCH", the output columns include: - // 1) the partition columns; - // 2) the ordering columns; - // 3) the columns defined in the measures clause; - // 4) any remaining columns defined of the input. - val fieldsAccessed = mutable.Set[Int]() - val resultExprs = - partitionKeys.asScala.map { case inputRef: RexInputRef => - fieldsAccessed += inputRef.getIndex - generateFieldAccess(input, eventNameTerm, inputRef.getIndex) - } ++ orderKeys.getFieldCollations.asScala.map { fieldCollation => - fieldsAccessed += fieldCollation.getFieldIndex - generateFieldAccess(input, eventNameTerm, fieldCollation.getFieldIndex) - } ++ (0 until input.getArity).filterNot(fieldsAccessed.contains).map { idx => - generateFieldAccess(input, eventNameTerm, idx) - } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => - generateExpression(measures.get(fieldName)) - } - - val resultExpression = generateResultExpression( - resultExprs, - returnType.typeInfo, - returnType.fieldNames) - - val resultCode = - s""" - |for (String $patternNameTerm : $patternNameListTerm) { - | $currPatternTerm = $patternNameTerm; - | $listTypeTerm $eventNameListTerm = ($listTypeTerm) $input1Term.get($patternNameTerm); - | if ($eventNameListTerm != null) { - | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { - | $currEventTerm = $eventNameTerm; - | ${resultExpression.code} - | $collectorTerm.collect(${resultExpression.resultTerm}); - | } - | } - |} - |$currPatternTerm = null; - |$currEventTerm = null; - |""".stripMargin - - GeneratedExpression("", "false", resultCode, null) - } - - override def visitCall(call: RexCall): GeneratedExpression = { - val resultType = FlinkTypeFactory.toTypeInfo(call.getType) - call.getOperator match { - case PREV => - val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generatePrev( - call.operands.get(0), - count, - resultType) - - case NEXT | CLASSIFIER | MATCH_NUMBER => - throw new CodeGenException(s"Unsupported call: $call") - - case FIRST | LAST => - val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generateFirstLast( - call.operands.get(0), - count, - resultType, - running = true, - call.getOperator == FIRST) - - case RUNNING | FINAL => - generateRunningFinal( - call.operands.get(0), - resultType, - call.getOperator == RUNNING) - - case _ => super.visitCall(call) - } - } - - private def generatePrev( - rexNode: RexNode, - count: Int, - resultType: TypeInformation[_]) - : GeneratedExpression = { - rexNode match { - case patternFieldRef: RexPatternFieldRef => - if (count == 0 && patternFieldRef.getAlpha == patternName.get) { - // return current one - return visitInputRef(patternFieldRef) - } - - val listName = newName("patternEvents") - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val indexTerm = newName("eventIndex") - val visitedEventNumberTerm = newName("visitedEventNumber") - val eventTerm = newName("event") - val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - - val patternNamesToVisit = patternNames - .take(patternNames.indexOf(patternFieldRef.getAlpha) + 1) - .reverse - def findEventByPhysicalPosition: String = { - val init: String = - s""" - |java.util.List $listName = new java.util.ArrayList(); - |""".stripMargin - - val getResult: String = { - for (tmpPatternName <- patternNamesToVisit) yield - s""" - |for ($eventTypeTerm $eventTerm : $contextTerm - | .getEventsForPattern("$tmpPatternName")) { - | $listName.add($eventTerm); - |} - | - |$indexTerm = $listName.size() - ($count - $visitedEventNumberTerm); - |if ($indexTerm >= 0) { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) $listName.get($indexTerm)) - | .getField(${patternFieldRef.getIndex}); - | $nullTerm = false; - | break; - |} - | - |$visitedEventNumberTerm += $listName.size(); - |$listName.clear(); - |""".stripMargin - }.mkString("\n") - - s""" - |$init - |$getResult - |""".stripMargin - } - - val resultCode = - s""" - |int $visitedEventNumberTerm = 0; - |int $indexTerm; - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |do { - | $findEventByPhysicalPosition - |} while (false); - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generatePrev( - operand, - count, - FlinkTypeFactory.toTypeInfo(operand.getType)) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } - - private def generateFirstLast( - rexNode: RexNode, - count: Int, - resultType: TypeInformation[_], - running: Boolean, - first: Boolean) - : GeneratedExpression = { - rexNode match { - case patternFieldRef: RexPatternFieldRef => - - val eventNameTerm = newName("event") - val resultTerm = newName("result") - val listName = newName("patternEvents") - val nullTerm = newName("isNull") - val patternNameTerm = newName("patternName") - val eventNameListTerm = newName("eventNameList") - val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val listTypeTerm = classOf[java.util.List[_]].getCanonicalName - - def findEventByLogicalPosition: String = { - val init = - s""" - |java.util.List $listName = new java.util.ArrayList(); - |""".stripMargin - - val findEventsByPatterName = if (generateCondition) { - s""" - |for ($eventTypeTerm $eventNameTerm : $contextTerm - | .getEventsForPattern("${patternFieldRef.getAlpha}")) { - | $listName.add($eventNameTerm); - |} - |""".stripMargin - } else { - s""" - |for (String $patternNameTerm : $patternNameListTerm) { - | if ($patternNameTerm.equals("${patternFieldRef.getAlpha}") || - | ${patternFieldRef.getAlpha.equals("*")}) { - | boolean skipLoop = false; - | $listTypeTerm $eventNameListTerm = - | ($listTypeTerm) $input1Term.get($patternNameTerm); - | if ($eventNameListTerm != null) { - | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { - | $listName.add($eventNameTerm); - | if ($running && $eventNameTerm == $currEventTerm) { - | skipLoop = true; - | break; - | } - | } - | } - | - | if (skipLoop) { - | break; - | } - | } - | - | if ($running && $patternNameTerm.equals($currPatternTerm)) { - | break; - | } - |} - |""".stripMargin - } - - val getResult = - s""" - |if ($listName.size() > $count) { - | if ($first) { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) - | $listName.get($count)) - | .getField(${patternFieldRef.getIndex}); - | } else { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) - | $listName.get($listName.size() - $count - 1)) - | .getField(${patternFieldRef.getIndex}); - | } - | $nullTerm = false; - |} - |""".stripMargin - - s""" - |$init - |$findEventsByPatterName - |$getResult - |""".stripMargin - } - - val resultCode = - s""" - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |$findEventByLogicalPosition - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generateFirstLast( - operand, - count, - FlinkTypeFactory.toTypeInfo(operand.getType), - running, - first) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } - - /* - - - */ - private def generateRunningFinal( - rexNode: RexNode, - resultType: TypeInformation[_], - running: Boolean): GeneratedExpression - = { - rexNode match { - case _: RexPatternFieldRef => - generateFirstLast(rexNode, 0, resultType, running, first = false) - - case rexCall: RexCall if rexCall.getOperator == FIRST || rexCall.getOperator == LAST => - val countLiteral = rexCall.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generateFirstLast( - rexCall.operands.get(0), - count, - resultType, - running, - rexCall.getOperator == FIRST) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generateRunningFinal( - operand, - FlinkTypeFactory.toTypeInfo(operand.getType), - running) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala deleted file mode 100644 index 3551d80ed..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala +++ /dev/null @@ -1,100 +0,0 @@ -package org.apache.flink.table.codegen - -import org.apache.flink.api.common.functions -import org.apache.flink.api.common.functions.Function -import org.apache.flink.api.common.io.InputFormat -import org.apache.flink.api.common.typeinfo.TypeInformation - -/** - * Describes a generated expression. - * - * @param resultTerm term to access the result of the expression - * @param nullTerm boolean term that indicates if expression is null - * @param code code necessary to produce resultTerm and nullTerm - * @param resultType type of the resultTerm - * @param literal flag to indicate a constant expression do not reference input and can thus - * be used in the member area (e.g. as constructor parameter of a reusable - * instance) - */ -case class GeneratedExpression( - resultTerm: String, - nullTerm: String, - code: String, - resultType: TypeInformation[_], - literal: Boolean = false) - -object GeneratedExpression { - val ALWAYS_NULL = "true" - val NEVER_NULL = "false" - val NO_CODE = "" -} - -/** - * Describes a generated [[functions.Function]] - * - * @param name class name of the generated Function. - * @param returnType the type information of the result type - * @param code code of the generated Function. - * @tparam F type of function - * @tparam T type of function - */ -case class GeneratedFunction[F <: Function, T <: Any]( - name: String, - returnType: TypeInformation[T], - code: String) - -/** - * Describes a generated aggregate helper function - * - * @param name class name of the generated Function. - * @param code code of the generated Function. - */ -case class GeneratedAggregationsFunction( - name: String, - code: String) - -/** - * Describes a generated [[InputFormat]]. - * - * @param name class name of the generated input function. - * @param returnType the type information of the result type - * @param code code of the generated Function. - * @tparam F type of function - * @tparam T type of function - */ -case class GeneratedInput[F <: InputFormat[_, _], T <: Any]( - name: String, - returnType: TypeInformation[T], - code: String) - -/** - * Describes a generated [[org.apache.flink.util.Collector]]. - * - * @param name class name of the generated Collector. - * @param code code of the generated Collector. - */ -case class GeneratedCollector(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.pattern.conditions.IterativeCondition]]. - * - * @param name class name of the generated IterativeCondition. - * @param code code of the generated IterativeCondition. - */ -case class GeneratedIterativeCondition(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.PatternSelectFunction]]. - * - * @param name class name of the generated PatternSelectFunction. - * @param code code of the generated PatternSelectFunction. - */ -case class GeneratedPatternSelectFunction(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.PatternFlatSelectFunction]]. - * - * @param name class name of the generated PatternFlatSelectFunction. - * @param code code of the generated PatternFlatSelectFunction. - */ -case class GeneratedPatternFlatSelectFunction(name: String, code: String) diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala deleted file mode 100644 index 10f60ba88..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala +++ /dev/null @@ -1,309 +0,0 @@ -package org.apache.flink.table.plan.nodes.datastream - -import java.math.{BigDecimal => JBigDecimal} -import java.util - -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} -import org.apache.calcite.rel._ -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rex._ -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable._ -import org.apache.flink.cep.{CEP, PatternStream} -import org.apache.flink.cep.pattern.Pattern -import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.streaming.api.windowing.time.Time -import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.RowtimeProcessFunction -import org.apache.flink.table.runtime.cepmatch.{ConvertToRow, MatchUtil} -import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.types.Row - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer - -/** - * Flink RelNode which matches along with LogicalMatch. - */ -class DataStreamMatch( - cluster: RelOptCluster, - traitSet: RelTraitSet, - input: RelNode, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode, - schema: RowSchema, - inputSchema: RowSchema) - extends SingleRel(cluster, traitSet, input) - with DataStreamRel { - - override def deriveRowType(): RelDataType = schema.relDataType - - override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { - new DataStreamMatch( - cluster, - traitSet, - inputs.get(0), - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval, - schema, - inputSchema) - } - - override def toString: String = { - s"Match(${ - if (!partitionKeys.isEmpty) { - s"PARTITION BY: ${partitionKeys.toArray.map(_.toString).mkString(", ")}, " - } else { - "" - } - }${ - if (!orderKeys.getFieldCollations.isEmpty) { - s"ORDER BY: ${orderKeys.getFieldCollations.asScala.map { - x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName - }.mkString(", ")}, " - } else { - "" - } - }${ - if (!measures.isEmpty) { - s"MEASURES: ${measures.asScala.map { - case (k, v) => s"${v.toString} AS $k" - }.mkString(", ")}, " - } else { - "" - } - }${ - if (allRows) { - s"ALL ROWS PER MATCH, " - } else { - s"ONE ROW PER MATCH, " - } - }${ - s"${after.toString}, " - }${ - s"PATTERN: (${pattern.toString})" - }${ - if (interval != null) { - s"WITHIN INTERVAL: $interval, " - } else { - s", " - } - }${ - if (!subsets.isEmpty) { - s"SUBSET: ${subsets.asScala.map { - case (k, v) => s"$k = (${v.toArray.mkString(", ")})" - }.mkString(", ")}, " - } else { - "" - } - }${ - s"DEFINE: ${patternDefinitions.asScala.map { - case (k, v) => s"$k AS ${v.toString}" - }.mkString(", ")}" - })" - } - - override def explainTerms(pw: RelWriter): RelWriter = { - pw.input("input", getInput()) - .itemIf("partitionBy", - partitionKeys.toArray.map(_.toString).mkString(", "), - !partitionKeys.isEmpty) - .itemIf("orderBy", - orderKeys.getFieldCollations.asScala.map { - x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName - }.mkString(", "), - !orderKeys.getFieldCollations.isEmpty) - .itemIf("measures", - measures.asScala.map { case (k, v) => s"${v.toString} AS $k"}.mkString(", "), - !measures.isEmpty) - .item("allrows", allRows) - .item("after", after.toString) - .item("pattern", pattern.toString) - .itemIf("within interval", - if (interval != null) { - interval.toString - } else { - null - }, - interval != null) - .itemIf("subset", - subsets.asScala.map { case (k, v) => s"$k = (${v.toArray.mkString(", ")})"}.mkString(", "), - !subsets.isEmpty) - .item("define", - patternDefinitions.asScala.map { case (k, v) => s"$k AS ${v.toString}"}.mkString(", ")) - } - - override def translateToPlan( - tableEnv: StreamTableEnvironment, - queryConfig: StreamQueryConfig): DataStream[CRow] = { - - val config = tableEnv.config - val inputTypeInfo = inputSchema.typeInfo - - val crowInput: DataStream[CRow] = getInput - .asInstanceOf[DataStreamRel] - .translateToPlan(tableEnv, queryConfig) - - val rowtimeFields = inputSchema.relDataType - .getFieldList.asScala - .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) - - //主要是针对rowtime做处理,对存在rowtime的将其复制到StreamRecord的时间戳字段内 - val timestampedInput = if (rowtimeFields.nonEmpty) { - // copy the rowtime field into the StreamRecord timestamp field - val timeIdx = rowtimeFields.head.getIndex - - crowInput - .process(new RowtimeProcessFunction(timeIdx, CRowTypeInfo(inputTypeInfo))) - .setParallelism(crowInput.getParallelism) - .name(s"rowtime field: (${rowtimeFields.head})") - } else { - crowInput - } - - val inputDS: DataStream[Row] = timestampedInput - .map(new ConvertToRow) - .setParallelism(timestampedInput.getParallelism) - .name("ConvertToRow") - .returns(inputTypeInfo) - - def translatePattern( - rexNode: RexNode, - currentPattern: Pattern[Row, Row], - patternNames: ListBuffer[String]): Pattern[Row, Row] = rexNode match { - //Conditions - case literal: RexLiteral => - val patternName = literal.getValue3.toString - patternNames += patternName - val newPattern = next(currentPattern, patternName) - - val patternDefinition = patternDefinitions.get(patternName) - if (patternDefinition != null) { - val condition = MatchUtil.generateIterativeCondition( - config, - inputSchema, - patternName, - patternNames, - patternDefinition, - inputTypeInfo) - - newPattern.where(condition) - } else { - newPattern - } - - case call: RexCall => - - call.getOperator match { - case PATTERN_CONCAT => - val left = call.operands.get(0) - val right = call.operands.get(1) - translatePattern(right, - translatePattern(left, currentPattern, patternNames), - patternNames) - // Quantifiers - case PATTERN_QUANTIFIER => - val name = call.operands.get(0).asInstanceOf[RexLiteral] - val newPattern = translatePattern(name, currentPattern, patternNames) - - val startNum = call.operands.get(1).asInstanceOf[RexLiteral] - .getValue3.asInstanceOf[JBigDecimal].intValue() - val endNum = call.operands.get(2).asInstanceOf[RexLiteral] - .getValue3.asInstanceOf[JBigDecimal].intValue() - - if (startNum == 0 && endNum == -1) { // zero or more - newPattern.oneOrMore().optional().consecutive() - } else if (startNum == 1 && endNum == -1) { // one or more - newPattern.oneOrMore().consecutive() - } else if (startNum == 0 && endNum == 1) { // optional - newPattern.optional() - } else if (endNum != -1) { // times - newPattern.times(startNum, endNum).consecutive() - } else { // times or more - newPattern.timesOrMore(startNum).consecutive() - } - - case PATTERN_ALTER => - throw TableException("Currently, CEP doesn't support branching patterns.") - - case PATTERN_PERMUTE => - throw TableException("Currently, CEP doesn't support PERMUTE patterns.") - - case PATTERN_EXCLUDE => - throw TableException("Currently, CEP doesn't support '{-' '-}' patterns.") - } - - case _ => - throw TableException("") - } - - val patternNames: ListBuffer[String] = ListBuffer() - val cepPattern = translatePattern(pattern, null, patternNames) - if (interval != null) { - val intervalLiteral = interval.asInstanceOf[RexLiteral] - val intervalValue = interval.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long]) - val intervalMs: Long = intervalLiteral.getTypeName match { - case INTERVAL_YEAR | INTERVAL_YEAR_MONTH | INTERVAL_MONTH => - // convert from months to milliseconds, suppose 1 month = 30 days - intervalValue * 30L * 24 * 3600 * 1000 - case _ => intervalValue - } - - cepPattern.within(Time.milliseconds(intervalMs)) - } - - val patternStream: PatternStream[Row] = CEP.pattern[Row](inputDS, cepPattern) - - val outTypeInfo = CRowTypeInfo(schema.typeInfo) - if (allRows) { - val patternFlatSelectFunction = - MatchUtil.generatePatternFlatSelectFunction( - config, - schema, - patternNames, - partitionKeys, - orderKeys, - measures, - inputTypeInfo) - patternStream.flatSelect[CRow](patternFlatSelectFunction, outTypeInfo) - } else { - val patternSelectFunction = - MatchUtil.generatePatternSelectFunction( - config, - schema, - patternNames, - partitionKeys, - measures, - inputTypeInfo) - patternStream.select[CRow](patternSelectFunction, outTypeInfo) - } - } - - private def next(currentPattern: Pattern[Row, Row], patternName: String): Pattern[Row, Row] = { - if (currentPattern == null) { - Pattern.begin(patternName) - } else { - currentPattern.next(patternName) - } - } -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala deleted file mode 100644 index 693e4d314..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala +++ /dev/null @@ -1,115 +0,0 @@ -package org.apache.flink.table.plan.nodes.logical - -import java.util - -import org.apache.calcite.plan._ -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.convert.ConverterRule -import org.apache.calcite.rel.core.Match -import org.apache.calcite.rel.logical.LogicalMatch -import org.apache.calcite.rel.{RelCollation, RelNode} -import org.apache.calcite.rex.RexNode -import org.apache.flink.table.plan.nodes.FlinkConventions - - -class FlinkLogicalMatch( - cluster: RelOptCluster, - traitSet: RelTraitSet, - input: RelNode, - rowType: RelDataType, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, _ <: util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode) - extends Match( - cluster, - traitSet, - input, - rowType, - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval) - with FlinkLogicalRel { - - override def copy( - input: RelNode, - rowType: RelDataType, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, _ <: util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode): Match = { - new FlinkLogicalMatch( - cluster, - traitSet, - input, - rowType, - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval) - } -} - -private class FlinkLogicalMatchConverter - extends ConverterRule( - classOf[LogicalMatch], - Convention.NONE, - FlinkConventions.LOGICAL, - "FlinkLogicalMatchConverter") { - - override def convert(rel: RelNode): RelNode = { - val logicalMatch = rel.asInstanceOf[LogicalMatch] - val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) - val newInput = RelOptRule.convert(logicalMatch.getInput, FlinkConventions.LOGICAL) - - new FlinkLogicalMatch( - rel.getCluster, - traitSet, - newInput, - logicalMatch.getRowType, - logicalMatch.getPattern, - logicalMatch.isStrictStart, - logicalMatch.isStrictEnd, - logicalMatch.getPatternDefinitions, - logicalMatch.getMeasures, - logicalMatch.getAfter, - logicalMatch.getSubsets, - logicalMatch.isAllRows, - logicalMatch.getPartitionKeys, - logicalMatch.getOrderKeys, - logicalMatch.getInterval) - } -} - -object FlinkLogicalMatch { - val CONVERTER: ConverterRule = new FlinkLogicalMatchConverter() -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala deleted file mode 100644 index 5a0ff9d65..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ /dev/null @@ -1,209 +0,0 @@ -package org.apache.flink.table.plan.rules - -import org.apache.calcite.rel.core.RelFactories -import org.apache.calcite.rel.rules._ -import org.apache.calcite.tools.{RuleSet, RuleSets} -import org.apache.flink.table.plan.rules.common._ -import org.apache.flink.table.plan.rules.logical._ -import org.apache.flink.table.plan.rules.dataSet._ -import org.apache.flink.table.plan.rules.datastream.{DataStreamMatchRule, _} -import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalMatch, _} - -object FlinkRuleSets { - - /** - * Convert sub-queries before query decorrelation. - */ - val TABLE_SUBQUERY_RULES: RuleSet = RuleSets.ofList( - SubQueryRemoveRule.FILTER, - SubQueryRemoveRule.PROJECT, - SubQueryRemoveRule.JOIN) - - /** - * Convert table references before query decorrelation. - */ - val TABLE_REF_RULES: RuleSet = RuleSets.ofList( - TableScanRule.INSTANCE, - EnumerableToLogicalTableScan.INSTANCE) - - val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList( - - // push a filter into a join - FilterJoinRule.FILTER_ON_JOIN, - // push filter into the children of a join - FilterJoinRule.JOIN, - // push filter through an aggregation - FilterAggregateTransposeRule.INSTANCE, - // push filter through set operation - FilterSetOpTransposeRule.INSTANCE, - // push project through set operation - ProjectSetOpTransposeRule.INSTANCE, - - // aggregation and projection rules - AggregateProjectMergeRule.INSTANCE, - AggregateProjectPullUpConstantsRule.INSTANCE, - // push a projection past a filter or vice versa - ProjectFilterTransposeRule.INSTANCE, - FilterProjectTransposeRule.INSTANCE, - // push a projection to the children of a join - // push all expressions to handle the time indicator correctly - new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), - // merge projections - ProjectMergeRule.INSTANCE, - // remove identity project - ProjectRemoveRule.INSTANCE, - // reorder sort and projection - SortProjectTransposeRule.INSTANCE, - ProjectSortTransposeRule.INSTANCE, - - // join rules - JoinPushExpressionsRule.INSTANCE, - - // remove union with only a single child - UnionEliminatorRule.INSTANCE, - // convert non-all union into all-union + distinct - UnionToDistinctRule.INSTANCE, - - // remove aggregation if it does not aggregate and input is already distinct - AggregateRemoveRule.INSTANCE, - // push aggregate through join - AggregateJoinTransposeRule.EXTENDED, - // aggregate union rule - AggregateUnionAggregateRule.INSTANCE, - // expand distinct aggregate to normal aggregate with groupby - AggregateExpandDistinctAggregatesRule.JOIN, - - // reduce aggregate functions like AVG, STDDEV_POP etc. - AggregateReduceFunctionsRule.INSTANCE, - - // remove unnecessary sort rule - SortRemoveRule.INSTANCE, - - // prune empty results rules - PruneEmptyRules.AGGREGATE_INSTANCE, - PruneEmptyRules.FILTER_INSTANCE, - PruneEmptyRules.JOIN_LEFT_INSTANCE, - PruneEmptyRules.JOIN_RIGHT_INSTANCE, - PruneEmptyRules.PROJECT_INSTANCE, - PruneEmptyRules.SORT_INSTANCE, - PruneEmptyRules.UNION_INSTANCE, - - // calc rules - FilterCalcMergeRule.INSTANCE, - ProjectCalcMergeRule.INSTANCE, - FilterToCalcRule.INSTANCE, - ProjectToCalcRule.INSTANCE, - CalcMergeRule.INSTANCE, - - // scan optimization - PushProjectIntoTableSourceScanRule.INSTANCE, - PushFilterIntoTableSourceScanRule.INSTANCE, - - // unnest rule - LogicalUnnestRule.INSTANCE, - - // translate to flink logical rel nodes - FlinkLogicalAggregate.CONVERTER, - FlinkLogicalWindowAggregate.CONVERTER, - FlinkLogicalOverWindow.CONVERTER, - FlinkLogicalCalc.CONVERTER, - FlinkLogicalCorrelate.CONVERTER, - FlinkLogicalIntersect.CONVERTER, - FlinkLogicalJoin.CONVERTER, - FlinkLogicalMinus.CONVERTER, - FlinkLogicalSort.CONVERTER, - FlinkLogicalUnion.CONVERTER, - FlinkLogicalValues.CONVERTER, - FlinkLogicalTableSourceScan.CONVERTER, - FlinkLogicalTableFunctionScan.CONVERTER, - FlinkLogicalNativeTableScan.CONVERTER, - FlinkLogicalMatch.CONVERTER - ) - - - /** - * RuleSet to normalize plans for batch / DataSet execution - */ - val DATASET_NORM_RULES: RuleSet = RuleSets.ofList( - // simplify expressions rules - ReduceExpressionsRule.FILTER_INSTANCE, - ReduceExpressionsRule.PROJECT_INSTANCE, - ReduceExpressionsRule.CALC_INSTANCE, - ReduceExpressionsRule.JOIN_INSTANCE, - ProjectToWindowRule.PROJECT, - - // Transform grouping sets - DecomposeGroupingSetRule.INSTANCE, - // Transform window to LogicalWindowAggregate - DataSetLogicalWindowAggregateRule.INSTANCE, - WindowPropertiesRule.INSTANCE, - WindowPropertiesHavingRule.INSTANCE - ) - - /** - * RuleSet to optimize plans for batch / DataSet execution - */ - val DATASET_OPT_RULES: RuleSet = RuleSets.ofList( - // translate to Flink DataSet nodes - DataSetWindowAggregateRule.INSTANCE, - DataSetAggregateRule.INSTANCE, - DataSetDistinctRule.INSTANCE, - DataSetCalcRule.INSTANCE, - DataSetJoinRule.INSTANCE, - DataSetSingleRowJoinRule.INSTANCE, - DataSetScanRule.INSTANCE, - DataSetUnionRule.INSTANCE, - DataSetIntersectRule.INSTANCE, - DataSetMinusRule.INSTANCE, - DataSetSortRule.INSTANCE, - DataSetValuesRule.INSTANCE, - DataSetCorrelateRule.INSTANCE, - BatchTableSourceScanRule.INSTANCE - ) - - /** - * RuleSet to normalize plans for stream / DataStream execution - */ - val DATASTREAM_NORM_RULES: RuleSet = RuleSets.ofList( - // Transform window to LogicalWindowAggregate - DataStreamLogicalWindowAggregateRule.INSTANCE, - WindowPropertiesRule.INSTANCE, - WindowPropertiesHavingRule.INSTANCE, - - // simplify expressions rules - ReduceExpressionsRule.FILTER_INSTANCE, - ReduceExpressionsRule.PROJECT_INSTANCE, - ReduceExpressionsRule.CALC_INSTANCE, - ProjectToWindowRule.PROJECT - ) - - /** - * RuleSet to optimize plans for stream / DataStream execution - */ - val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( - // translate to DataStream nodes - DataStreamSortRule.INSTANCE, - DataStreamGroupAggregateRule.INSTANCE, - DataStreamOverAggregateRule.INSTANCE, - DataStreamGroupWindowAggregateRule.INSTANCE, - DataStreamCalcRule.INSTANCE, - DataStreamScanRule.INSTANCE, - DataStreamUnionRule.INSTANCE, - DataStreamValuesRule.INSTANCE, - DataStreamCorrelateRule.INSTANCE, - DataStreamWindowJoinRule.INSTANCE, - StreamTableSourceScanRule.INSTANCE, - DataStreamMatchRule.INSTANCE - ) - - /** - * RuleSet to decorate plans for stream / DataStream execution - */ - val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList( - // retraction rules - DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE, - DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE, - DataStreamRetractionRules.ACCMODE_INSTANCE - ) - -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala deleted file mode 100644 index 8bb052504..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala +++ /dev/null @@ -1,46 +0,0 @@ -package org.apache.flink.table.plan.rules.datastream - -import org.apache.calcite.plan.{RelOptRule, RelTraitSet} -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.convert.ConverterRule -import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.nodes.datastream.DataStreamMatch -import org.apache.flink.table.plan.nodes.logical.FlinkLogicalMatch -import org.apache.flink.table.plan.schema.RowSchema - -class DataStreamMatchRule - extends ConverterRule( - classOf[FlinkLogicalMatch], - FlinkConventions.LOGICAL, - FlinkConventions.DATASTREAM, - "DataStreamMatchRule") { - - override def convert(rel: RelNode): RelNode = { - val logicalMatch: FlinkLogicalMatch = rel.asInstanceOf[FlinkLogicalMatch] - val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) - val convertInput: RelNode = - RelOptRule.convert(logicalMatch.getInput, FlinkConventions.DATASTREAM) - - new DataStreamMatch( - rel.getCluster, - traitSet, - convertInput, - logicalMatch.getPattern, - logicalMatch.isStrictStart, - logicalMatch.isStrictEnd, - logicalMatch.getPatternDefinitions, - logicalMatch.getMeasures, - logicalMatch.getAfter, - logicalMatch.getSubsets, - logicalMatch.isAllRows, - logicalMatch.getPartitionKeys, - logicalMatch.getOrderKeys, - logicalMatch.getInterval, - new RowSchema(logicalMatch.getRowType), - new RowSchema(logicalMatch.getInput.getRowType)) - } -} - -object DataStreamMatchRule { - val INSTANCE: RelOptRule = new DataStreamMatchRule -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala deleted file mode 100644 index cfaf5fe81..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala +++ /dev/null @@ -1,14 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import org.apache.flink.api.common.functions.MapFunction -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row - -/** - * MapFunction convert CRow to Row. - */ -class ConvertToRow extends MapFunction[CRow, Row] { - override def map(value: CRow): Row = { - value.row - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala deleted file mode 100644 index 04a365ebb..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.types.Row -import org.slf4j.LoggerFactory - -/** - * IterativeConditionRunner with [[Row]] value. - */ -class IterativeConditionRunner( - name: String, - code: String) - extends IterativeCondition[Row] - with Compiler[IterativeCondition[Row]]{ - - val LOG = LoggerFactory.getLogger(this.getClass) - - // IterativeCondition will be serialized as part of state, - // so make function as transient to avoid ClassNotFoundException when restore state, - // see FLINK-6939 for details - @transient private var function: IterativeCondition[Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling IterativeCondition: $name \n\n Code:\n$code") - // We cannot get user's classloader currently, see FLINK-6938 for details - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating IterativeCondition.") - function = clazz.newInstance() - } - - override def filter(value: Row, ctx: IterativeCondition.Context[Row]): Boolean = { - - if (function == null) { - init() - } - - function.filter(value, ctx) - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala deleted file mode 100644 index e0af24e93..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala +++ /dev/null @@ -1,99 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rex.RexNode -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.codegen.MatchCodeGenerator -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row - -/** - * An util class to generate match functions. - * 1。IterativeCondition - * 2。PatternSelectFunction - * 3。PatternFlatSelectFunction - */ -object MatchUtil { - - private[flink] def generateIterativeCondition( - config: TableConfig, - inputType: RowSchema, - patternName: String, - patternNames: Seq[String], - patternDefinition: RexNode, - inputTypeInfo: TypeInformation[_]): IterativeCondition[Row] = { - - val generator = new MatchCodeGenerator( - config, false, inputTypeInfo, patternNames, true, Some(patternName)) - val condition = generator.generateExpression(patternDefinition) - val body = - s""" - |${condition.code} - |return ${condition.resultTerm}; - |""".stripMargin - - val genCondition = generator.generateIterativeCondition("MatchRecognizeCondition", body) - new IterativeConditionRunner(genCondition.name, genCondition.code) - } - - private[flink] def generatePatternSelectFunction( - config: TableConfig, - returnType: RowSchema, - patternNames: Seq[String], - partitionKeys: util.List[RexNode], - measures: util.Map[String, RexNode], - inputTypeInfo: TypeInformation[_]): PatternSelectFunction[Row, CRow] = { - - val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) - - val resultExpression = generator.generateSelectOutputExpression( - partitionKeys, - measures, - returnType) - val body = - s""" - |${resultExpression.code} - |return ${resultExpression.resultTerm}; - |""".stripMargin - - generator.addReusableStatements() - val genFunction = generator.generatePatternSelectFunction( - "MatchRecognizePatternSelectFunction", - body) - new PatternSelectFunctionRunner(genFunction.name, genFunction.code) - } - - private[flink] def generatePatternFlatSelectFunction( - config: TableConfig, - returnType: RowSchema, - patternNames: Seq[String], - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - measures: util.Map[String, RexNode], - inputTypeInfo: TypeInformation[_]): PatternFlatSelectFunction[Row, CRow] = { - - val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) - - val resultExpression = generator.generateFlatSelectOutputExpression( - partitionKeys, - orderKeys, - measures, - returnType) - val body = - s""" - |${resultExpression.code} - |""".stripMargin - - generator.addReusableStatements() - val genFunction = generator.generatePatternFlatSelectFunction( - "MatchRecognizePatternFlatSelectFunction", - body) - new PatternFlatSelectFunctionRunner(genFunction.name, genFunction.code) - } -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala deleted file mode 100644 index f72296324..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala +++ /dev/null @@ -1,47 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.flink.cep.PatternFlatSelectFunction -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.table.runtime.CRowWrappingCollector -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row -import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory - -/** - * PatternFlatSelectFunctionRunner with [[Row]] input and [[CRow]] output. - */ -class PatternFlatSelectFunctionRunner( - name: String, - code: String) - extends PatternFlatSelectFunction[Row, CRow] - with Compiler[PatternFlatSelectFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) - - private var cRowWrapper: CRowWrappingCollector = _ - - private var function: PatternFlatSelectFunction[Row, Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling PatternFlatSelectFunction: $name \n\n Code:\n$code") - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating PatternFlatSelectFunction.") - function = clazz.newInstance() - - this.cRowWrapper = new CRowWrappingCollector() - } - - override def flatSelect( - pattern: util.Map[String, util.List[Row]], - out: Collector[CRow]): Unit = { - if (function == null) { - init() - } - - cRowWrapper.out = out - function.flatSelect(pattern, cRowWrapper) - } -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala deleted file mode 100644 index d84c9e61d..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala +++ /dev/null @@ -1,45 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.flink.cep.PatternSelectFunction -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row -import org.slf4j.LoggerFactory - -/** - * PatternSelectFunctionRunner with [[Row]] input and [[CRow]] output. - */ -class PatternSelectFunctionRunner( - name: String, - code: String) - extends PatternSelectFunction[Row, CRow] - with Compiler[PatternSelectFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) - - private var outCRow: CRow = _ - - private var function: PatternSelectFunction[Row, Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling PatternSelectFunction: $name \n\n Code:\n$code") - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating PatternSelectFunction.") - function = clazz.newInstance() - } - - override def select(pattern: util.Map[String, util.List[Row]]): CRow = { - if (outCRow == null) { - outCRow = new CRow(null, true) - } - - if (function == null) { - init() - } - - outCRow.row = function.select(pattern) - outCRow - } -} diff --git a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala deleted file mode 100644 index f89c41d71..000000000 --- a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala +++ /dev/null @@ -1,534 +0,0 @@ -package org.apache.flink.table.validate - -import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes, SqlTypeTransforms} -import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable} -import org.apache.calcite.sql._ -import org.apache.flink.table.api._ -import org.apache.flink.table.expressions._ -import org.apache.flink.table.functions.sql.ScalarSqlFunctions -import org.apache.flink.table.functions.utils.{AggSqlFunction, ScalarSqlFunction, TableSqlFunction} -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} - -import _root_.scala.collection.JavaConversions._ -import _root_.scala.collection.mutable -import _root_.scala.util.{Failure, Success, Try} - -/** - * A catalog for looking up (user-defined) functions, used during validation phases - * of both Table API and SQL API. - */ -class FunctionCatalog { - - private val functionBuilders = mutable.HashMap.empty[String, Class[_]] - private val sqlFunctions = mutable.ListBuffer[SqlFunction]() - - def registerFunction(name: String, builder: Class[_]): Unit = - functionBuilders.put(name.toLowerCase, builder) - - def registerSqlFunction(sqlFunction: SqlFunction): Unit = { - sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName) - sqlFunctions += sqlFunction - } - - def getUserDefinedFunctions: Seq[String] = { - sqlFunctions.map(_.getName) - } - - def getSqlOperatorTable: SqlOperatorTable = - ChainedSqlOperatorTable.of( - new BasicOperatorTable(), - new ListSqlOperatorTable(sqlFunctions) - ) - - /** - * Lookup and create an expression if we find a match. - */ - def lookupFunction(name: String, children: Seq[Expression]): Expression = { - val funcClass = functionBuilders - .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name")) - - // Instantiate a function using the provided `children` - funcClass match { - - // user-defined scalar function call - case sf if classOf[ScalarFunction].isAssignableFrom(sf) => - val scalarSqlFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[ScalarSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined scalar function: $name")) - .asInstanceOf[ScalarSqlFunction] - ScalarFunctionCall(scalarSqlFunction.getScalarFunction, children) - - // user-defined table function call - case tf if classOf[TableFunction[_]].isAssignableFrom(tf) => - val tableSqlFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined table function: $name")) - .asInstanceOf[TableSqlFunction] - val typeInfo = tableSqlFunction.getRowTypeInfo - val function = tableSqlFunction.getTableFunction - TableFunctionCall(name, function, children, typeInfo) - - // user-defined aggregate function call - case af if classOf[AggregateFunction[_, _]].isAssignableFrom(af) => - val aggregateFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[AggSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined table function: $name")) - .asInstanceOf[AggSqlFunction] - val function = aggregateFunction.getFunction - val returnType = aggregateFunction.returnType - val accType = aggregateFunction.accType - AggFunctionCall(function, returnType, accType, children) - - // general expression call - case expression if classOf[Expression].isAssignableFrom(expression) => - // try to find a constructor accepts `Seq[Expression]` - Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match { - case Success(seqCtor) => - Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(e) => throw new ValidationException(e.getMessage) - } - case Failure(_) => - Try(funcClass.getDeclaredConstructor(classOf[Expression], classOf[Seq[_]])) match { - case Success(ctor) => - Try(ctor.newInstance(children.head, children.tail).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(e) => throw new ValidationException(e.getMessage) - } - case Failure(_) => - val childrenClass = Seq.fill(children.length)(classOf[Expression]) - // try to find a constructor matching the exact number of children - Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match { - case Success(ctor) => - Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(exception) => throw ValidationException(exception.getMessage) - } - case Failure(_) => - throw ValidationException( - s"Invalid number of arguments for function $funcClass") - } - } - } - case _ => - throw ValidationException("Unsupported function.") - } - } - - /** - * Drop a function and return if the function existed. - */ - def dropFunction(name: String): Boolean = - functionBuilders.remove(name.toLowerCase).isDefined - - /** - * Drop all registered functions. - */ - def clear(): Unit = functionBuilders.clear() -} - -object FunctionCatalog { - - val builtInFunctions: Map[String, Class[_]] = Map( - - // logic - "and" -> classOf[And], - "or" -> classOf[Or], - "not" -> classOf[Not], - "equals" -> classOf[EqualTo], - "greaterThan" -> classOf[GreaterThan], - "greaterThanOrEqual" -> classOf[GreaterThanOrEqual], - "lessThan" -> classOf[LessThan], - "lessThanOrEqual" -> classOf[LessThanOrEqual], - "notEquals" -> classOf[NotEqualTo], - "in" -> classOf[In], - "isNull" -> classOf[IsNull], - "isNotNull" -> classOf[IsNotNull], - "isTrue" -> classOf[IsTrue], - "isFalse" -> classOf[IsFalse], - "isNotTrue" -> classOf[IsNotTrue], - "isNotFalse" -> classOf[IsNotFalse], - "if" -> classOf[If], - - // aggregate functions - "avg" -> classOf[Avg], - "count" -> classOf[Count], - "max" -> classOf[Max], - "min" -> classOf[Min], - "sum" -> classOf[Sum], - "sum0" -> classOf[Sum0], - "stddevPop" -> classOf[StddevPop], - "stddevSamp" -> classOf[StddevSamp], - "varPop" -> classOf[VarPop], - "varSamp" -> classOf[VarSamp], - "collect" -> classOf[Collect], - - // string functions - "charLength" -> classOf[CharLength], - "initCap" -> classOf[InitCap], - "like" -> classOf[Like], - "concat" -> classOf[Plus], - "lower" -> classOf[Lower], - "lowerCase" -> classOf[Lower], - "similar" -> classOf[Similar], - "substring" -> classOf[Substring], - "trim" -> classOf[Trim], - "upper" -> classOf[Upper], - "upperCase" -> classOf[Upper], - "position" -> classOf[Position], - "overlay" -> classOf[Overlay], - "concat" -> classOf[Concat], - "concat_ws" -> classOf[ConcatWs], - "lpad" -> classOf[Lpad], - "rpad" -> classOf[Rpad], - - // math functions - "plus" -> classOf[Plus], - "minus" -> classOf[Minus], - "divide" -> classOf[Div], - "times" -> classOf[Mul], - "abs" -> classOf[Abs], - "ceil" -> classOf[Ceil], - "exp" -> classOf[Exp], - "floor" -> classOf[Floor], - "log10" -> classOf[Log10], - "ln" -> classOf[Ln], - "power" -> classOf[Power], - "mod" -> classOf[Mod], - "sqrt" -> classOf[Sqrt], - "minusPrefix" -> classOf[UnaryMinus], - "sin" -> classOf[Sin], - "cos" -> classOf[Cos], - "tan" -> classOf[Tan], - "cot" -> classOf[Cot], - "asin" -> classOf[Asin], - "acos" -> classOf[Acos], - "atan" -> classOf[Atan], - "degrees" -> classOf[Degrees], - "radians" -> classOf[Radians], - "sign" -> classOf[Sign], - "round" -> classOf[Round], - "pi" -> classOf[Pi], - "e" -> classOf[E], - "rand" -> classOf[Rand], - "randInteger" -> classOf[RandInteger], - "bin" -> classOf[Bin], - - // temporal functions - "extract" -> classOf[Extract], - "currentDate" -> classOf[CurrentDate], - "currentTime" -> classOf[CurrentTime], - "currentTimestamp" -> classOf[CurrentTimestamp], - "localTime" -> classOf[LocalTime], - "localTimestamp" -> classOf[LocalTimestamp], - "quarter" -> classOf[Quarter], - "temporalOverlaps" -> classOf[TemporalOverlaps], - "dateTimePlus" -> classOf[Plus], - "dateFormat" -> classOf[DateFormat], - - // item - "at" -> classOf[ItemAt], - - // cardinality - "cardinality" -> classOf[Cardinality], - - // array - "array" -> classOf[ArrayConstructor], - "element" -> classOf[ArrayElement], - - // map - "map" -> classOf[MapConstructor], - - // row - "row" -> classOf[RowConstructor], - - // window properties - "start" -> classOf[WindowStart], - "end" -> classOf[WindowEnd], - - // ordering - "asc" -> classOf[Asc], - "desc" -> classOf[Desc], - - // crypto hash - "md5" -> classOf[Md5], - "sha1" -> classOf[Sha1], - "sha256" -> classOf[Sha256] - ) - - /** - * Create a new function catalog with built-in functions. - */ - def withBuiltIns: FunctionCatalog = { - val catalog = new FunctionCatalog() - builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) } - catalog - } -} - -class BasicOperatorTable extends ReflectiveSqlOperatorTable { - - /** - * List of supported SQL operators / functions. - * - * This list should be kept in sync with [[SqlStdOperatorTable]]. - */ - private val builtInSqlOperators: Seq[SqlOperator] = Seq( - // SET OPERATORS - SqlStdOperatorTable.UNION, - SqlStdOperatorTable.UNION_ALL, - SqlStdOperatorTable.EXCEPT, - SqlStdOperatorTable.EXCEPT_ALL, - SqlStdOperatorTable.INTERSECT, - SqlStdOperatorTable.INTERSECT_ALL, - // BINARY OPERATORS - SqlStdOperatorTable.AND, - SqlStdOperatorTable.AS, - SqlStdOperatorTable.CONCAT, - SqlStdOperatorTable.DIVIDE, - SqlStdOperatorTable.DIVIDE_INTEGER, - SqlStdOperatorTable.DOT, - SqlStdOperatorTable.EQUALS, - SqlStdOperatorTable.GREATER_THAN, - SqlStdOperatorTable.IS_DISTINCT_FROM, - SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, - SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, - SqlStdOperatorTable.LESS_THAN, - SqlStdOperatorTable.LESS_THAN_OR_EQUAL, - SqlStdOperatorTable.MINUS, - SqlStdOperatorTable.MULTIPLY, - SqlStdOperatorTable.NOT_EQUALS, - SqlStdOperatorTable.OR, - SqlStdOperatorTable.PLUS, - SqlStdOperatorTable.DATETIME_PLUS, - // POSTFIX OPERATORS - SqlStdOperatorTable.DESC, - SqlStdOperatorTable.NULLS_FIRST, - SqlStdOperatorTable.IS_NOT_NULL, - SqlStdOperatorTable.IS_NULL, - SqlStdOperatorTable.IS_NOT_TRUE, - SqlStdOperatorTable.IS_TRUE, - SqlStdOperatorTable.IS_NOT_FALSE, - SqlStdOperatorTable.IS_FALSE, - SqlStdOperatorTable.IS_NOT_UNKNOWN, - SqlStdOperatorTable.IS_UNKNOWN, - // PREFIX OPERATORS - SqlStdOperatorTable.NOT, - SqlStdOperatorTable.UNARY_MINUS, - SqlStdOperatorTable.UNARY_PLUS, - // GROUPING FUNCTIONS - SqlStdOperatorTable.GROUP_ID, - SqlStdOperatorTable.GROUPING, - SqlStdOperatorTable.GROUPING_ID, - // AGGREGATE OPERATORS - SqlStdOperatorTable.SUM, - SqlStdOperatorTable.SUM0, - SqlStdOperatorTable.COUNT, - SqlStdOperatorTable.COLLECT, - SqlStdOperatorTable.MIN, - SqlStdOperatorTable.MAX, - SqlStdOperatorTable.AVG, - SqlStdOperatorTable.STDDEV_POP, - SqlStdOperatorTable.STDDEV_SAMP, - SqlStdOperatorTable.VAR_POP, - SqlStdOperatorTable.VAR_SAMP, - // ARRAY OPERATORS - SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, - SqlStdOperatorTable.ELEMENT, - // MAP OPERATORS - SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR, - // ARRAY MAP SHARED OPERATORS - SqlStdOperatorTable.ITEM, - SqlStdOperatorTable.CARDINALITY, - // SPECIAL OPERATORS - SqlStdOperatorTable.ROW, - SqlStdOperatorTable.OVERLAPS, - SqlStdOperatorTable.LITERAL_CHAIN, - SqlStdOperatorTable.BETWEEN, - SqlStdOperatorTable.SYMMETRIC_BETWEEN, - SqlStdOperatorTable.NOT_BETWEEN, - SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN, - SqlStdOperatorTable.NOT_LIKE, - SqlStdOperatorTable.LIKE, - SqlStdOperatorTable.NOT_SIMILAR_TO, - SqlStdOperatorTable.SIMILAR_TO, - SqlStdOperatorTable.CASE, - SqlStdOperatorTable.REINTERPRET, - SqlStdOperatorTable.EXTRACT, - SqlStdOperatorTable.IN, - // FUNCTIONS - SqlStdOperatorTable.SUBSTRING, - SqlStdOperatorTable.OVERLAY, - SqlStdOperatorTable.TRIM, - SqlStdOperatorTable.POSITION, - SqlStdOperatorTable.CHAR_LENGTH, - SqlStdOperatorTable.CHARACTER_LENGTH, - SqlStdOperatorTable.UPPER, - SqlStdOperatorTable.LOWER, - SqlStdOperatorTable.INITCAP, - SqlStdOperatorTable.POWER, - SqlStdOperatorTable.SQRT, - SqlStdOperatorTable.MOD, - SqlStdOperatorTable.LN, - SqlStdOperatorTable.LOG10, - SqlStdOperatorTable.ABS, - SqlStdOperatorTable.EXP, - SqlStdOperatorTable.NULLIF, - SqlStdOperatorTable.COALESCE, - SqlStdOperatorTable.FLOOR, - SqlStdOperatorTable.CEIL, - SqlStdOperatorTable.LOCALTIME, - SqlStdOperatorTable.LOCALTIMESTAMP, - SqlStdOperatorTable.CURRENT_TIME, - SqlStdOperatorTable.CURRENT_TIMESTAMP, - SqlStdOperatorTable.CURRENT_DATE, - ScalarSqlFunctions.DATE_FORMAT, - SqlStdOperatorTable.CAST, - SqlStdOperatorTable.EXTRACT, - SqlStdOperatorTable.QUARTER, - SqlStdOperatorTable.SCALAR_QUERY, - SqlStdOperatorTable.EXISTS, - SqlStdOperatorTable.SIN, - SqlStdOperatorTable.COS, - SqlStdOperatorTable.TAN, - SqlStdOperatorTable.COT, - SqlStdOperatorTable.ASIN, - SqlStdOperatorTable.ACOS, - SqlStdOperatorTable.ATAN, - SqlStdOperatorTable.DEGREES, - SqlStdOperatorTable.RADIANS, - SqlStdOperatorTable.SIGN, - SqlStdOperatorTable.ROUND, - SqlStdOperatorTable.PI, - ScalarSqlFunctions.E, - SqlStdOperatorTable.RAND, - SqlStdOperatorTable.RAND_INTEGER, - ScalarSqlFunctions.CONCAT, - ScalarSqlFunctions.CONCAT_WS, - ScalarSqlFunctions.BIN, - SqlStdOperatorTable.TIMESTAMP_ADD, - ScalarSqlFunctions.LOG, - ScalarSqlFunctions.LPAD, - ScalarSqlFunctions.RPAD, - ScalarSqlFunctions.MD5, - ScalarSqlFunctions.SHA1, - ScalarSqlFunctions.SHA256, - - // EXTENSIONS - BasicOperatorTable.TUMBLE, - BasicOperatorTable.HOP, - BasicOperatorTable.SESSION, - BasicOperatorTable.TUMBLE_START, - BasicOperatorTable.TUMBLE_END, - BasicOperatorTable.HOP_START, - BasicOperatorTable.HOP_END, - BasicOperatorTable.SESSION_START, - BasicOperatorTable.SESSION_END, - BasicOperatorTable.TUMBLE_PROCTIME, - BasicOperatorTable.TUMBLE_ROWTIME, - BasicOperatorTable.HOP_PROCTIME, - BasicOperatorTable.HOP_ROWTIME, - BasicOperatorTable.SESSION_PROCTIME, - BasicOperatorTable.SESSION_ROWTIME, - - // MATCH_RECOGNIZE - SqlStdOperatorTable.FIRST, - SqlStdOperatorTable.LAST, - SqlStdOperatorTable.PREV, - SqlStdOperatorTable.NEXT, - SqlStdOperatorTable.CLASSIFIER, - SqlStdOperatorTable.MATCH_NUMBER, - SqlStdOperatorTable.FINAL, - SqlStdOperatorTable.RUNNING - ) - - builtInSqlOperators.foreach(register) -} - -object BasicOperatorTable { - - /** - * We need custom group auxiliary functions in order to support nested windows. - */ - - val TUMBLE: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.TUMBLE, - null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - TUMBLE_START, - TUMBLE_END, - TUMBLE_ROWTIME, - TUMBLE_PROCTIME) - } - val TUMBLE_START: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_START) - val TUMBLE_END: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_END) - val TUMBLE_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "TUMBLE_ROWTIME", - SqlKind.OTHER_FUNCTION, - TUMBLE, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - TUMBLE.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val TUMBLE_PROCTIME: SqlGroupedWindowFunction = - TUMBLE.auxiliary("TUMBLE_PROCTIME", SqlKind.OTHER_FUNCTION) - - val HOP: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.HOP, - null, - OperandTypes.or( - OperandTypes.DATETIME_INTERVAL_INTERVAL, - OperandTypes.DATETIME_INTERVAL_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - HOP_START, - HOP_END, - HOP_ROWTIME, - HOP_PROCTIME) - } - val HOP_START: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_START) - val HOP_END: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_END) - val HOP_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "HOP_ROWTIME", - SqlKind.OTHER_FUNCTION, - HOP, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - HOP.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val HOP_PROCTIME: SqlGroupedWindowFunction = HOP.auxiliary("HOP_PROCTIME", SqlKind.OTHER_FUNCTION) - - val SESSION: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.SESSION, - null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - SESSION_START, - SESSION_END, - SESSION_ROWTIME, - SESSION_PROCTIME) - } - val SESSION_START: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_START) - val SESSION_END: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_END) - val SESSION_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "SESSION_ROWTIME", - SqlKind.OTHER_FUNCTION, - SESSION, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - SESSION.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val SESSION_PROCTIME: SqlGroupedWindowFunction = - SESSION.auxiliary("SESSION_PROCTIME", SqlKind.OTHER_FUNCTION) - -} 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 4513649fa..f02a81819 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,10 +18,6 @@ package com.dtstack.flink.sql.launcher; -import com.dtstack.flink.sql.util.PluginUtil; -import com.dtstack.flink.yarn.JobParameter; -import com.dtstack.flink.yarn.YarnClusterConfiguration; -import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; @@ -34,25 +30,16 @@ import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import java.io.File; import java.net.InetSocketAddress; -import java.net.URLDecoder; import java.util.*; import com.dtstack.flink.sql.ClusterMode; -import org.apache.hadoop.yarn.exceptions.YarnException; -import java.io.IOException; -import java.util.stream.Collectors; -import java.util.stream.Stream; import com.dtstack.flink.sql.options.LauncherOptions; -import static java.util.Objects.requireNonNull; /** * @author sishu.yss @@ -64,7 +51,7 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); } else if(mode.equals(ClusterMode.yarn.name())) { - return createYarnClient(launcherOptions,mode); + return createYarnClient(launcherOptions); } throw new IllegalArgumentException("Unsupported cluster client type: "); } @@ -81,167 +68,65 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio return clusterClient; } - public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { + public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration flinkConf = GlobalConfiguration.loadConfiguration(flinkConfDir); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); - YarnConfiguration yarnConf; if(StringUtils.isNotBlank(yarnConfDir)) { + try { - flinkConf.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); - FileSystem.initialize(flinkConf); - - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - yarnConf = loadYarnConfiguration(yarnConfDir); - - YarnClient yarnClient = YarnClient.createYarnClient(); - haYarnConf(yarnConf); - yarnClient.init(yarnConf); - yarnClient.start(); - - String confProp = launcherOptions.getConfProp(); - confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); - System.out.println("confProp="+confProp); - Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); - - ApplicationId applicationId = null; - ClusterClient clusterClient = null; - if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode - applicationId = getYarnClusterApplicationId(yarnClient); - System.out.println("applicationId="+applicationId.toString()); - - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( - flinkConf, yarnConf, ".", yarnClient, false); - clusterClient = clusterDescriptor.retrieve(applicationId); - - System.out.println("applicationId="+applicationId.toString()+" has retrieve!"); - } else {//on yarn per-job mode - applicationId = createApplication(yarnClient); - System.out.println("applicationId="+applicationId.toString()); - - YarnClusterConfiguration clusterConf = getYarnClusterConfiguration(flinkConf,yarnConf,flinkConfDir); - //jobmanager+taskmanager param - JobParameter appConf = new JobParameter(confProperties); - - com.dtstack.flink.yarn.YarnClusterDescriptor clusterDescriptor = new com.dtstack.flink.yarn.YarnClusterDescriptor( - clusterConf, yarnClient, appConf,applicationId, launcherOptions.getName(),null ); - clusterClient = clusterDescriptor.deploy(); - - System.out.println("applicationId="+applicationId.toString()+" has deploy!"); + config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + FileSystem.initialize(config); + + 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; } - clusterClient.setDetached(true); - yarnClient.stop(); - return clusterClient; - } - } catch(Exception e) { - throw new RuntimeException(e); - } - } - throw new UnsupportedOperationException("Haven't been developed yet!"); - } - private static YarnConfiguration loadYarnConfiguration(String yarnConfDir) - { - org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - hadoopConf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem"); - - Stream.of("yarn-site.xml", "core-site.xml", "hdfs-site.xml").forEach(file -> { - File site = new File(requireNonNull(yarnConfDir, "ENV HADOOP_CONF_DIR is not setting"), file); - if (site.exists() && site.isFile()) { - hadoopConf.addResource(new org.apache.hadoop.fs.Path(site.toURI())); - } - else { - throw new RuntimeException(site + " not exists"); - } - }); - - YarnConfiguration yarnConf = new YarnConfiguration(hadoopConf); - // try (PrintWriter pw = new PrintWriter(new FileWriter(yarnSite))) { //write local file - // yarnConf.writeXml(pw); - // } - return yarnConf; - } - - public static YarnClusterConfiguration getYarnClusterConfiguration(Configuration flinkConf,YarnConfiguration yarnConf,String flinkConfDir) - { - Path flinkJar = new Path(getFlinkJarFile(flinkConfDir).toURI()); - @SuppressWarnings("ConstantConditions") final Set resourcesToLocalize = Stream - .of("flink-conf.yaml", "log4j.properties") - .map(x -> new Path(new File(flinkConfDir, x).toURI())) - .collect(Collectors.toSet()); - - return new YarnClusterConfiguration(flinkConf, yarnConf, "", flinkJar, resourcesToLocalize); - } + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; + } - public static final String FLINK_DIST = "flink-dist"; - private static File getFlinkJarFile(String flinkConfDir) - { - String errorMessage = "error not search " + FLINK_DIST + "*.jar"; - File[] files = requireNonNull(new File(flinkConfDir, "/../lib").listFiles(), errorMessage); - Optional file = Arrays.stream(files) - .filter(f -> f.getName().startsWith(FLINK_DIST)).findFirst(); - return file.orElseThrow(() -> new IllegalArgumentException(errorMessage)); - } + 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(); + } - private static ApplicationId createApplication(YarnClient yarnClient)throws IOException, YarnException { - YarnClientApplication app = yarnClient.createApplication(); - return app.getApplicationSubmissionContext().getApplicationId(); - } - private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient) throws Exception{ - 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; - } + 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; + } catch(Exception e) { + throw new RuntimeException(e); } - } - if(StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); - } - return applicationId; + throw new UnsupportedOperationException("Haven't been developed yet!"); } - /** - * 处理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; - } + } \ No newline at end of file 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 1832cf717..b6a8c5c64 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 @@ -67,23 +67,24 @@ public static void main(String[] args) throws Exception { LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + String[] localArgs = argList.toArray(new String[argList.size()]); if(mode.equals(ClusterMode.local.name())) { - Main.main(args); + Main.main(localArgs); }else{ String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); - PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), args); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), localArgs); 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, launcherOptions.getDefaultParallelism()); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, 1); PerJobSubmitter.submit(launcherOptions, jobGraph); } else { ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - clusterClient.run(program, launcherOptions.getDefaultParallelism()); + clusterClient.run(program, 1); clusterClient.shutdown(); System.exit(0); } From 7fb8317b5c0a70d82031e15ce528cfef2dea1596 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 27 Feb 2019 20:52:38 +0800 Subject: [PATCH 75/78] Update kafkaSource.md --- docs/kafkaSource.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index a78239421..98423c787 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -203,7 +203,7 @@ create table kafka_stream( sex ) Group BY T.sex, - TUMBLE(ctime, INTERVAL '1' MINUTE); + TUMBLE(rowtime, INTERVAL '1' MINUTE); -- 对input_view中输出的数据做计算 CREATE VIEW view2 ( cnt, @@ -214,7 +214,7 @@ create table kafka_stream( T.sex from input_view - Group BY sex, TUMBLE(ctime, INTERVAL '1' MINUTE); + Group BY sex, TUMBLE(rowtime, INTERVAL '1' MINUTE); -- 使用解析出的格式化数据进行计算,并将结果输出到RDS中 insert into rds_sink SELECT From 6952f533604c45efceb5d9fd12f473a23ff2cbb9 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 27 Feb 2019 20:54:43 +0800 Subject: [PATCH 76/78] Update kafkaSource.md --- docs/kafkaSource.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 98423c787..315aa9820 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -203,7 +203,7 @@ create table kafka_stream( sex ) Group BY T.sex, - TUMBLE(rowtime, INTERVAL '1' MINUTE); + TUMBLE(ROWTIME, INTERVAL '1' MINUTE); -- 对input_view中输出的数据做计算 CREATE VIEW view2 ( cnt, @@ -214,7 +214,7 @@ create table kafka_stream( T.sex from input_view - Group BY sex, TUMBLE(rowtime, INTERVAL '1' MINUTE); + Group BY sex, TUMBLE(ROWTIME, INTERVAL '1' MINUTE); -- 使用解析出的格式化数据进行计算,并将结果输出到RDS中 insert into rds_sink SELECT From 2c8409cbd584d67adde28af03e68882084413ccf Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 27 Feb 2019 21:04:40 +0800 Subject: [PATCH 77/78] Lex.MYSQL --- core/src/main/java/com/dtstack/flink/sql/Main.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 11917c4cd..aa05528ce 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -104,6 +104,11 @@ public class Main { private static final int delayInterval = 10; //sec + private static org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + public static void main(String[] args) throws Exception { Options options = new Options(); @@ -192,11 +197,6 @@ public static void main(String[] args) throws Exception { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); - - org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser - .configBuilder() - .setLex(Lex.MYSQL) - .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); From 1b2512ed3eb810278f02ee4611272f60cdecc2a2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 28 Feb 2019 14:24:21 +0800 Subject: [PATCH 78/78] null --- .../kafka/deserialization/CustomerCommonDeserialization.java | 1 + .../kafka/deserialization/CustomerCommonDeserialization.java | 5 +++-- .../kafka/deserialization/CustomerCommonDeserialization.java | 5 +++-- .../kafka/deserialization/CustomerCommonDeserialization.java | 5 +++-- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index c92ce4aa4..4f0e9ec32 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -79,6 +79,7 @@ public boolean isEndOfStream(Row nextElement) { return false; } + @Override public TypeInformation getProducedType() { TypeInformation[] types = new TypeInformation[]{ TypeExtractor.createTypeInfo(String.class), diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index 4a235e296..524b0cffb 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -71,8 +71,8 @@ public Row deserialize(byte[] messageKey, byte[] message, String topic, int part } numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); + if(message!=null){numInBytes.inc(message.length);} + if(messageKey!=null){numInBytes.inc(messageKey.length);} try { Row row = Row.of( @@ -105,6 +105,7 @@ public boolean isEndOfStream(Row nextElement) { return false; } + @Override public TypeInformation getProducedType() { TypeInformation[] types = new TypeInformation[]{ TypeExtractor.createTypeInfo(String.class), diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index 3502c959c..9c3b9b03f 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -73,8 +73,8 @@ public Row deserialize(byte[] messageKey, byte[] message, String topic, int part } numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); + if(message!=null){numInBytes.inc(message.length);} + if(messageKey!=null){numInBytes.inc(messageKey.length);} try { Row row = Row.of( @@ -107,6 +107,7 @@ public boolean isEndOfStream(Row nextElement) { return false; } + @Override public TypeInformation getProducedType() { TypeInformation[] types = new TypeInformation[]{ TypeExtractor.createTypeInfo(String.class), diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index 7fc891638..a032e6a82 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -74,8 +74,8 @@ public Row deserialize(byte[] messageKey, byte[] message, String topic, int part } numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); + if(message!=null){numInBytes.inc(message.length);} + if(messageKey!=null){numInBytes.inc(messageKey.length);} try { Row row = Row.of( @@ -108,6 +108,7 @@ public boolean isEndOfStream(Row nextElement) { return false; } + @Override public TypeInformation getProducedType() { TypeInformation[] types = new TypeInformation[]{ TypeExtractor.createTypeInfo(String.class),