diff --git a/hetu-docs/en/admin/extension-execution-planner.md b/hetu-docs/en/admin/extension-execution-planner.md index 45a1109ab1ed86e36fa3cfb3bde1537455d6d9be..f1dd46f96c8c1e3a68d2485a632742f8f79e865b 100644 --- a/hetu-docs/en/admin/extension-execution-planner.md +++ b/hetu-docs/en/admin/extension-execution-planner.md @@ -2,12 +2,18 @@ This section describes how to add an extension physical execution planner in openLooKeng. With the extension physical execution planner, openLooKeng can utilize other operator acceleration libraries to speed up the execution of SQL statements. ## Configuration -To enable extension physical execution feature, the following configs must be added in +First, you should get the below jar package and c++ libraries from links: + +- `boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.jar`: [Link](https://github.com/kunpengcompute/boostkit-bigdata/releases/download/v1.3.0/boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.zip) + +- `libboostkit-omniop-xx-1.0.0-aarch.so`: [Link](https://www.hikunpeng.com/en/developer/boostkit/big-data?acclerated=3) + +Then, to enable extension physical execution feature, the following configs must be added in `config.properties`: ``` properties extension_execution_planner_enabled=true -extension_execution_planner_jar_path=file:///xxPath/omni-openLooKeng-adapter-1.6.1-SNAPSHOT.jar +extension_execution_planner_jar_path=file:///xxPath/boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.jar extension_execution_planner_class_path=nova.hetu.olk.OmniLocalExecutionPlanner ``` diff --git a/hetu-docs/zh/admin/extension-execution-planner.md b/hetu-docs/zh/admin/extension-execution-planner.md index 52c54736f4a59fd90ae2c2aaea92917f8cc24566..b3250ee35bcea83775d6c3644be4036d25ce10aa 100644 --- a/hetu-docs/zh/admin/extension-execution-planner.md +++ b/hetu-docs/zh/admin/extension-execution-planner.md @@ -2,11 +2,16 @@ 本节介绍openLooKeng如何添加扩展物理执行计划。通过物理执行计划的扩展,openLooKeng可以使用其他算子加速库来加速SQL语句的执行。 ## 配置 -在配置文件`config.properties`增加如下配置: +首先,需要从以下链接获取相关jar包和c++库: +- `boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.jar`: [Link](https://github.com/kunpengcompute/boostkit-bigdata/releases/download/v1.3.0/boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.zip) + +- `libboostkit-omniop-xx-1.0.0-aarch.so`: [Link](https://www.hikunpeng.com/zh/developer/boostkit/big-data?acclerated=3) + +然后,在配置文件`config.properties`增加如下配置: ``` properties extension_execution_planner_enabled=true -extension_execution_planner_jar_path=file:///xxPath/omni-openLooKeng-adapter-1.6.1-SNAPSHOT.jar +extension_execution_planner_jar_path=file:///xxPath/boostkit-omniop-openlookeng-1.6.1-1.0.0-aarch64.jar extension_execution_planner_class_path=nova.hetu.olk.OmniLocalExecutionPlanner ``` diff --git a/hetu-docs/zh/connector/kafka.md b/hetu-docs/zh/connector/kafka.md index ab1f61156cce82436cef283829cf1c08e178700d..35e87302e55bdfa911d51c99153d6cf00d0c2f7b 100644 --- a/hetu-docs/zh/connector/kafka.md +++ b/hetu-docs/zh/connector/kafka.md @@ -29,16 +29,23 @@ kafka.nodes=host1:port,host2:port 配置属性包括: -| 属性名称| 说明| -|:----------|:----------| -| `kafka.table-names`| 目录提供的所有表列表| -| `kafka.default-schema`| 表的默认模式名| -| `kafka.nodes`| Kafka集群节点列表| -| `kafka.connect-timeout`| 连接Kafka集群超时| -| `kafka.buffer-size`| Kafka读缓冲区大小| -| `kafka.table-description-dir`| 包含主题描述文件的目录| -| `kafka.hide-internal-columns`| 控制内部列是否是表模式的一部分| - +| 属性名称| 说明 | +|:----------|:-----------------------------------| +| `kafka.table-names`| 目录提供的所有表列表 | +| `kafka.default-schema`| 表的默认模式名 | +| `kafka.nodes`| Kafka集群节点列表 | +| `kafka.connect-timeout`| 连接Kafka集群超时 | +| `kafka.buffer-size`| Kafka读缓冲区大小 | +| `kafka.table-description-dir`| 包含主题描述文件的目录 | +| `kafka.hide-internal-columns`| 控制内部列是否是表模式的一部分 | +| `kerberos.on`| 是否开启Kerberos认证 | +| `user.password.auth.on`| 是否开启kafka用户密码认证 | +| `sasl.jaas.config`| 认证相关信息 | +| `java.security.krb5.conf`| krb5.conf文件路径 | +| `group.id`| kafka的groupID | +| `security.protocol`| Kafka的安全认证协议 | +| `sasl.mechanism`| sasl机制 | +| `sasl.kerberos.service.name`| kafka服务运行时的kerberos principal name | ### `kafka.table-names` 此目录提供的所有表的逗号分隔列表。表名可以是非限定的(简单名称),并将被放入默认模式(见下文)中,或者用模式名称(`.`)限定。 @@ -87,6 +94,56 @@ openLooKeng必须仍然能够连接到群集的所有节点,即使这里只指 此属性是可选的;默认值为`true`。 +### `kerberos.on` + +是否开启kerberos认证,适用于开启了kerberos认证的集群,如果在运行presto-kafka中的测试包,请置为false,因为测试程序使用内嵌Kafka,不支持认证,且该项与`user.password.auth.on`仅能选择一个,若两者均为true,则`user.password.auth.on`将覆盖`kerberos.on`且会异常。 + +此属性是可选的;默认值为`false`。 + +### `user.password.auth.on` + +是否开启用户密码认证,适用于开启了用户密码认证的集群,如果在运行presto-kafka中的测试包,请置为false,因为测试程序使用内嵌Kafka,不支持认证,且该项与`kerberos.on`仅能选择一个,若两者均为true,则`user.password.auth.on`将覆盖`kerberos.on`且会异常。 +若使用用户密码认证,需要在jvm配置中增加-Djava.ext.dirs=$JAVA_HOME/jre/lib/ext:/Users/mac/apps/lib,其中/Users/mac/apps/lib下需要放置kafka-client.jar,否则会报出 loginModule not found的相关异常 +此属性是可选的;默认值为`false`。 + +### `sasl.jaas.config` + +Kafka的认证相关信息。 +对于kerberos认证,可能的值为:sasl.jaas.config= com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true useTicketCache=true serviceName=kafka keyTab=\"/Users/mac/Desktop/user01.keytab\" principal=\"user01@EXAMPLE.COM\"; +对于用户密码认证,可能的值为:sasl.jaas.config= org.apache.kafka.common.security.plain.PlainLoginModule required username=\"producer\" password=\"producerpwd\"; + +此属性是可选的;默认值为``。 + +### `java.security.krb5.conf` + +存放krb5.conf文件的路径,要注意全局配置中也需要配置此选项,例如部署后在jvm.config中配置,而在开发中需要在启动PrestoServer时使用"-D"参数配置。 + +此属性是可选的;默认值为``。 + +### `group.id` + +kafka的groupId。 + +此属性是可选的;默认值为``。 + +### `security.protocol` + +Kafka的安全协议。 + +此属性是可选的;默认值为`SASL_PLAINTEXT`。 + +### `sasl.mechanism` + +sasl机制,被用于客户端连接安全的机制。 + +此属性是可选的;默认值为`GSSAPI`。 + +### `sasl.kerberos.service.name` + +kafka运行时的kerberos principal name。 + +此属性是可选的;默认值为`kafka`。 + ## 内部列 对于每个已定义的表,连接器维护以下列: diff --git a/hetu-docs/zh/connector/mpp.md b/hetu-docs/zh/connector/mpp.md new file mode 100644 index 0000000000000000000000000000000000000000..c7633f7667f2e88337c9b886a5d9a5bdf62a916c --- /dev/null +++ b/hetu-docs/zh/connector/mpp.md @@ -0,0 +1,70 @@ + +# mpp连接器 + +本方案旨在通过GDS高速协议来提高openLookeng引擎读取GaussDB的性能。 + +目前对于GaussDB数据库的数据查询方式是通过JDBC Connector来实现的,和大部分关系型数据库连接类似。由于传统的MySQL等数据库存储的数据量小,因此通过JDBC方式来获取数据无可厚非。但是由于GaussDB是一种MPP类的分布式数据库,存储数据量大,且主要用于OLAP场景的分析,导致通过原生JDBC方式拉取数据的方式变得低效,因为它是一种单进单出的模式。 + +后来,社区也针对这种情况进行了基于JDBC的优化,例如增加了下推,在引擎端增加了多split并发,但是依然无法避免数据源端的单并发,性能瓶颈依然无法得到完全的解决,而且多并发还会带来多连接的情况,对数据库集群造成一定的压力。 + +本方案将会通过解决数据源端和引擎端并发问题来提高引擎查询的效率。 + +# mpp连接器设计思路 + +本方案将mpp类数据库的查询转换成对hive外表的查询,即将mpp类数据库的表数据快速导出到外部的分布式文件系统或分布式缓存,然后通过挂hive外表的方式通过hive connector来进行查询。本方案的主要权衡点在于导出数据的效率和mpp数据库通过单CN节点对外输出数据的效率的比较。随着查询数据量的递增,本方案的效率提高也会越来越高。 +更详细的内容见社区分享:https://mp.weixin.qq.com/s/Q-t592UerICHNXI63rhtPg + +## 配置 + +要配置mpp连接器,在`etc/catalog`中创建一个目录属性文件,例如`mpp.properties`,使用以下内容创建文件,并根据设置替换连接属性: + +``` properties +本方案本质上是将查询gaussdb数据库转换成查询hive,因此以下配置均基于此原理。 + +connector.name=mpp +# 配置用来做最后查询的hive仓库 +hive.metastore.uri=thrift://localhost:9083 + +etl-reuse=false #是否复用本次导数结果 + +#GDS baseinfo +#gds进程,基于postgres的fdw机制实现的一个快速导数进程,gaussdb官方插件 +gds-list=gsfs://localhost:port1|base_path #gds的ip和端口,以及该进程启动时候的basepath,多个gds进程可以通过逗号分隔 +aux-url=alluxio://localhost:19998 #alluxio的ip和端口 +base-aux=/gdsdata/ #alluxio中用来为gds导出数据服务的路径,可自定义 + +#hive info +# 用来进行创建外表等操作的hive仓库连接配置 +hive-user=username +hive-passwd=password +hive-db=xxx_db +hive-url=jdbc:hive2://localhost:10000/ + +# hive template +## 进行hive外表创建的相关SQL模板,一般无需更改 +hsql-drop=drop table if exists ${table_name} +hsql-create=CREATE EXTERNAL TABLE ${table_name} ( ${schema_info} ) COMMENT 'gds external table' ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\n' LOCATION '${pipe_to_aux_base_path}' + +# gsDB connection info +# 要查询的GaussDB数据库相关连接信息 +gs-driver=org.postgresql.Driver +gs-url=jdbc:postgresql://localhost:25308/schema +gs-user=user +gs-passwd=password + +# gaussdb template +# 利用gds触发导数的相SQL模板,一般无需更改 +gsql-create=create foreign table ${gaussdb_name}.ext_${table_name} ( ${schema_info} ) SERVER gsmpp_server OPTIONS ( LOCATION '${gds_foreign_location}', FORMAT 'text', DELIMITER E',', NULL '', encoding 'UTF-8', noescaping 'true', EOL E'\\n', out_filename_prefix '${table_name}') WRITE ONLY; +gsql-insert=insert into ${gaussdb_name}.ext_${table_name} select ${schema_info} from ${gaussdb_name}.${table_name}; +gsql-drop=drop foreign table if exists ${gaussdb_name}.ext_${table_name}; + +``` + +## 使用 + +mpp连接器会将查询转换为对所配置的gaussdb的查询,因此如果你要查询opengauss.testdb.usertbl,如果想通过mpp +connector进行快速查询,则可以写成: + + select * from mpp.testdb.usertbl; + +如果您对mpp connector有更多的需求和见解,欢迎提issue和pr。 \ No newline at end of file diff --git a/hetu-exchange-filesystem/pom.xml b/hetu-exchange-filesystem/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..27853241c5f182c6403032ca05b6e6136c6eed88 --- /dev/null +++ b/hetu-exchange-filesystem/pom.xml @@ -0,0 +1,205 @@ + + + + presto-root + io.hetu.core + 1.8.0-SNAPSHOT + + 4.0.0 + + hetu-exchange-filesystem + hetu-plugin + + + ${project.parent.basedir} + 2.17.2 + 13.0 + 2.0.1.Final + 1.10.19 + 6.10 + 0.8.2 + 3.1.1 + + + + + com.google.code.findbugs + jsr305 + true + + + javax.validation + validation-api + + + org.checkerframework + checker-qual + 2.5.2 + + + org.openjdk.jol + jol-core + + + commons-codec + commons-codec + runtime + + + + + io.hetu.core + presto-spi + provided + + + jackson-annotations + com.fasterxml.jackson.core + + + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + io.airlift + configuration + + + io.airlift + stats + + + io.airlift + bootstrap + + + com.google.guava + guava + + + com.google.inject + guice + + + + org.apache.logging.log4j + log4j-api + ${dep.log4j.version} + runtime + + + org.apache.logging.log4j + log4j-core + ${dep.log4j.version} + runtime + + + org.eclipse.jetty + jetty-webapp + 9.4.46.v20220331 + runtime + + + org.apache.hadoop + hadoop-distcp + runtime + + + jackson-databind + + + jackson-annotations + com.fasterxml.jackson.core + + + com.fasterxml.jackson.core + runtime + + + + + org.testng + testng + test + + + + org.weakref + jmxutils + ${dep.jmxutils.version} + compile + + + + io.hetu.core + presto-tests + test + + + plexus-cipher + org.sonatype.plexus + + + plexus-classworlds + org.codehaus.plexus + + + + + + io.hetu.core + hetu-metastore + test + + + + io.airlift + testing-mysql-server + test + + + + org.mockito + mockito-core + ${version.mockito-all} + test + + + io.hetu.core + presto-plugin-toolkit + + + + + + + org.gaul + modernizer-maven-plugin + 1.7.1 + + true + + + + + + \ No newline at end of file diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeReader.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeReader.java new file mode 100644 index 0000000000000000000000000000000000000000..d30eb624c33487bdbb582e6ec9317a5b6fdb64ef --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeReader.java @@ -0,0 +1,35 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.Slice; + +import java.io.Closeable; +import java.io.IOException; + +public interface ExchangeReader + extends Closeable +{ + Slice read() throws IOException; + + ListenableFuture isBlocked(); + + Long getRetainedSize(); + + boolean isFinished(); + + @Override + void close(); +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeSourceFile.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeSourceFile.java new file mode 100644 index 0000000000000000000000000000000000000000..f1111f1af90ed524e9e629626b0982cbfdbba78c --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeSourceFile.java @@ -0,0 +1,52 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import javax.annotation.concurrent.Immutable; +import javax.crypto.SecretKey; + +import java.net.URI; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +@Immutable +public class ExchangeSourceFile +{ + private final URI fileUri; + private final Optional secretKey; + private final long fileSize; + + public ExchangeSourceFile(URI fileUri, Optional secretKey, long fileSize) + { + this.fileUri = requireNonNull(fileUri, "fileUri is null"); + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + this.fileSize = fileSize; + } + + public URI getFileUri() + { + return fileUri; + } + + public Optional getSecretKey() + { + return secretKey; + } + + public long getFileSize() + { + return fileSize; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeWriter.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeWriter.java new file mode 100644 index 0000000000000000000000000000000000000000..53ed0df40903af91971b09a1ee28b2c49baabb1c --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExchangeWriter.java @@ -0,0 +1,28 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.slice.Slice; + +public interface ExchangeWriter +{ + ListenableFuture write(Slice slice); + + ListenableFuture finish(); + + ListenableFuture abort(); + + long getRetainedSize(); +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExecutionStats.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExecutionStats.java new file mode 100644 index 0000000000000000000000000000000000000000..6be9967048b85098caf435b3ae202ee443c21dd6 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/ExecutionStats.java @@ -0,0 +1,97 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.base.Stopwatch; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.stats.TimeStat; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class ExecutionStats +{ + private final TimeStat finished = new TimeStat(MILLISECONDS); + private final TimeStat failed = new TimeStat(MILLISECONDS); + + public T record(Supplier call) + { + Stopwatch stopwatch = Stopwatch.createStarted(); + try { + T result = call.get(); + finished.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + return result; + } + catch (Throwable t) { + failed.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + throw t; + } + } + + public CompletableFuture record(CompletableFuture future) + { + Stopwatch stopwatch = Stopwatch.createStarted(); + future.whenComplete((value, failure) -> { + if (failure == null) { + finished.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + } + else { + failed.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + } + }); + return future; + } + + public ListenableFuture record(ListenableFuture future) + { + Stopwatch stopwatch = Stopwatch.createStarted(); + Futures.addCallback(future, new FutureCallback() + { + @Override + public void onSuccess(@Nullable T result) + { + finished.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + } + + @Override + public void onFailure(Throwable t) + { + failed.add(stopwatch.elapsed(MILLISECONDS), MILLISECONDS); + } + }, directExecutor()); + return future; + } + + @Managed + @Nested + public TimeStat getFinished() + { + return finished; + } + + @Managed + @Nested + public TimeStat getFailed() + { + return failed; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileStatus.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileStatus.java new file mode 100644 index 0000000000000000000000000000000000000000..028ccbe426b9c315f822697bdd02f7fa0ef4e4bc --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileStatus.java @@ -0,0 +1,80 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.openjdk.jol.info.ClassLayout; + +import javax.annotation.concurrent.Immutable; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; + +@Immutable +public class FileStatus +{ + private static final int INSTANCE_SIZE = ClassLayout.parseClass(FileStatus.class).instanceSize(); + + private final String filePath; + private final long fileSize; + + @JsonCreator + public FileStatus(@JsonProperty("filePath") String filePath, @JsonProperty("fileSize") long fileSize) + { + this.filePath = filePath; + this.fileSize = fileSize; + } + + @JsonProperty + public String getFilePath() + { + return filePath; + } + + @JsonProperty + public long getFileSize() + { + return fileSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FileStatus that = (FileStatus) o; + return fileSize == that.fileSize && filePath.equals(that.filePath); + } + + @Override + public int hashCode() + { + return Objects.hash(filePath, fileSize); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("filePath", filePath) + .add("fileSize", fileSize) + .toString(); + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchange.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchange.java new file mode 100644 index 0000000000000000000000000000000000000000..d00fa60291e79bf6f3ac2dcab5f24f814e6fa9af --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchange.java @@ -0,0 +1,74 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.prestosql.spi.exchange.Exchange; +import io.prestosql.spi.exchange.ExchangeSinkHandle; +import io.prestosql.spi.exchange.ExchangeSinkInstanceHandle; +import io.prestosql.spi.exchange.ExchangeSourceHandle; +import io.prestosql.spi.exchange.ExchangeSourceSplitter; +import io.prestosql.spi.exchange.ExchangeSourceStatistics; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public class FileSystemExchange + implements Exchange +{ + @Override + public ExchangeSinkHandle addSink(int taskPartitionId) + { + return null; + } + + @Override + public void noMoreSinks() + { + } + + @Override + public ExchangeSinkInstanceHandle instantiateSink(ExchangeSinkHandle sinkHandle, int taskAttemptId) + { + return null; + } + + @Override + public void sinkFinished(ExchangeSinkInstanceHandle handle) + { + } + + @Override + public CompletableFuture> getSourceHandles() + { + return null; + } + + @Override + public ExchangeSourceSplitter split(ExchangeSourceHandle handle, long targetSizeInBytes) + { + return null; + } + + @Override + public ExchangeSourceStatistics getExchangeSourceStatistics(ExchangeSourceHandle handle) + { + return null; + } + + @Override + public void close() throws IOException + { + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeConfig.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..4ee7332772465f9afc7065d5dee563ff2264759e --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeConfig.java @@ -0,0 +1,173 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.collect.ImmutableList; +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.units.DataSize; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotEmpty; +import javax.validation.constraints.NotNull; + +import java.net.URI; +import java.util.Arrays; +import java.util.List; + +import static com.google.common.base.Strings.isNullOrEmpty; +import static io.airlift.units.DataSize.Unit.GIGABYTE; +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static io.hetu.core.plugin.exchange.filesystem.FileSystemExchangeManager.PATH_SEPARATOR; + +public class FileSystemExchangeConfig +{ + private List baseDirectories = ImmutableList.of(); + private boolean exchangeEncryptionEnabled = true; + + private DataSize maxPageStorageSize = new DataSize(16, MEGABYTE); + private int exchangeSinkBufferPoolMinSize = 10; + private int exchangeSinkBuffersPerPartition = 2; + private DataSize exchangeSinkMaxFileSize = new DataSize(1, GIGABYTE); + private int exchangeSourceConcurrentReaders = 4; + private int maxOutputPartitionCount = 50; + private int exchangeFileListingParallelism = 50; + + @NotNull + @NotEmpty(message = "At least one base directory needs to be configured") + public List getBaseDirectories() + { + return baseDirectories; + } + + @Config("exchange.base-directories") + @ConfigDescription("List of base directories separated by comma") + public FileSystemExchangeConfig setBaseDirectories(String baseDirectories) + { + if (baseDirectories != null && isNullOrEmpty(baseDirectories)) { + ImmutableList.Builder builder = ImmutableList.builder(); + Arrays.stream(baseDirectories.split(",")).forEach(dir -> { + if (!dir.endsWith(PATH_SEPARATOR)) { + dir += PATH_SEPARATOR; + } + builder.add(URI.create(dir)); + }); + this.baseDirectories = builder.build(); + } + return this; + } + + public boolean isExchangeEncryptionEnabled() + { + return exchangeEncryptionEnabled; + } + + @Config("exchange.encryption-enabled") + public FileSystemExchangeConfig setExchangeEncryptionEnabled(boolean exchangeEncryptionEnabled) + { + this.exchangeEncryptionEnabled = exchangeEncryptionEnabled; + return this; + } + + public DataSize getMaxPageStorageSize() + { + return maxPageStorageSize; + } + + @Config("exchange.max-page-storage-size") + @ConfigDescription("Max storage size of a page written to a sink, including the page itself and its size represented by an int") + public FileSystemExchangeConfig setMaxPageStorageSize(DataSize maxPageStorageSize) + { + this.maxPageStorageSize = maxPageStorageSize; + return this; + } + + public int getExchangeSinkBufferPoolMinSize() + { + return exchangeSinkBufferPoolMinSize; + } + + @Config("exchange.sink-buffer-pool-min-size") + public FileSystemExchangeConfig setExchangeSinkBufferPoolMinSize(int exchangeSinkBufferPoolMinSize) + { + this.exchangeSinkBufferPoolMinSize = exchangeSinkBufferPoolMinSize; + return this; + } + + @Min(2) + public int getExchangeSinkBuffersPerPartition() + { + return exchangeSinkBuffersPerPartition; + } + + @Config("exchange.sink-buffers-per-partition") + public FileSystemExchangeConfig setExchangeSinkBuffersPerPartition(int exchangeSinkBuffersPerPartition) + { + this.exchangeSinkBuffersPerPartition = exchangeSinkBuffersPerPartition; + return this; + } + + public DataSize getExchangeSinkMaxFileSize() + { + return exchangeSinkMaxFileSize; + } + + @Config("exchange.sink-max-file-size") + @ConfigDescription("Max size of files written by sinks") + public FileSystemExchangeConfig setExchangeSinkMaxFileSize(DataSize exchangeSinkMaxFileSize) + { + this.exchangeSinkMaxFileSize = exchangeSinkMaxFileSize; + return this; + } + + @Min(1) + public int getExchangeSourceConcurrentReaders() + { + return exchangeSourceConcurrentReaders; + } + + @Config("exchange.source-concurrent-readers") + public FileSystemExchangeConfig setExchangeSourceConcurrentReaders(int exchangeSourceConcurrentReaders) + { + this.exchangeSourceConcurrentReaders = exchangeSourceConcurrentReaders; + return this; + } + + @Min(1) + public int getMaxOutputPartitionCount() + { + return maxOutputPartitionCount; + } + + @Config("exchange.max-output-partition-count") + public FileSystemExchangeConfig setMaxOutputPartitionCount(int maxOutputPartitionCount) + { + this.maxOutputPartitionCount = maxOutputPartitionCount; + return this; + } + + @Min(1) + public int getExchangeFileListingParallelism() + { + return exchangeFileListingParallelism; + } + + @Config("exchange.file-listing-parallelism") + @ConfigDescription("Max parallelism of file listing calls when enumerating spooling files.") + public FileSystemExchangeConfig setExchangeFileListingParallelism(int exchangeFileListingParallelism) + { + this.exchangeFileListingParallelism = exchangeFileListingParallelism; + return this; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeErrorCode.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeErrorCode.java new file mode 100644 index 0000000000000000000000000000000000000000..f72cd0f55f0753cf9da7a8b283e2038625f2b87d --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeErrorCode.java @@ -0,0 +1,38 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.prestosql.spi.ErrorCode; +import io.prestosql.spi.ErrorCodeSupplier; +import io.prestosql.spi.ErrorType; + +import static io.prestosql.spi.ErrorType.USER_ERROR; + +public enum FileSystemExchangeErrorCode + implements ErrorCodeSupplier +{ + MAX_OUTPUT_PARTITION_COUNT_EXCEEDED(0, USER_ERROR),; + private final ErrorCode errorCode; + + FileSystemExchangeErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0510_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeFutures.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeFutures.java new file mode 100644 index 0000000000000000000000000000000000000000..e442229d315bba5a8846dc2557162d671812db5a --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeFutures.java @@ -0,0 +1,28 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.util.concurrent.ListenableFuture; + +public class FileSystemExchangeFutures +{ + private FileSystemExchangeFutures() + { + } + + public static ListenableFuture translateFailures(ListenableFuture listenableFuture) + { + return null; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManager.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManager.java new file mode 100644 index 0000000000000000000000000000000000000000..728b78d1b94577e4d8223d74542b8994c86603d8 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManager.java @@ -0,0 +1,48 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.prestosql.spi.exchange.Exchange; +import io.prestosql.spi.exchange.ExchangeContext; +import io.prestosql.spi.exchange.ExchangeManager; +import io.prestosql.spi.exchange.ExchangeSink; +import io.prestosql.spi.exchange.ExchangeSinkInstanceHandle; +import io.prestosql.spi.exchange.ExchangeSource; +import io.prestosql.spi.exchange.ExchangeSourceHandle; + +import java.util.List; + +public class FileSystemExchangeManager + implements ExchangeManager +{ + public static final String PATH_SEPARATOR = "/"; + + @Override + public Exchange createExchange(ExchangeContext context, int outputPartitionCount) + { + return null; + } + + @Override + public ExchangeSink createSink(ExchangeSinkInstanceHandle handle, boolean preserveRecordsOrder) + { + return null; + } + + @Override + public ExchangeSource createSource(List handles) + { + return null; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManagerFactory.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManagerFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..92b5cc77ed3afc660e03f850d55fcb04796b8e92 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeManagerFactory.java @@ -0,0 +1,73 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.prestosql.plugin.base.jmx.MBeanServerModule; +import io.prestosql.plugin.base.jmx.PrefixObjectNameGeneratorModule; +import io.prestosql.spi.exchange.ExchangeManager; +import io.prestosql.spi.exchange.ExchangeManagerFactory; +import io.prestosql.spi.exchange.ExchangeManagerHandleResolver; +import io.prestosql.spi.exchange.ExchangeSinkInstanceHandle; +import io.prestosql.spi.exchange.ExchangeSourceHandle; +import org.weakref.jmx.guice.MBeanModule; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class FileSystemExchangeManagerFactory + implements ExchangeManagerFactory +{ + @Override + public String getName() + { + return "filesystem"; + } + + @Override + public ExchangeManager create(Map config) + { + requireNonNull(config, "config is null"); + Bootstrap app = new Bootstrap( + new MBeanModule(), + new MBeanServerModule(), + new PrefixObjectNameGeneratorModule("io.hetu.core.plugin.exchange.filesystem", "hetu.core.plugin.exchange.filesystem"), + new FileSystemExchangeModule()); + Injector injector = app.doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + return injector.getInstance(FileSystemExchangeManager.class); + } + + @Override + public ExchangeManagerHandleResolver getHandleResolver() + { + return new ExchangeManagerHandleResolver() + { + @Override + public Class getExchangeSinkInstanceHandleClass() + { + return FileSystemExchangeSinkInstanceHandle.class; + } + + @Override + public Class getExchangeSourceHandleHandleClass() + { + return FileSystemExchangeSourceHandle.class; + } + }; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeModule.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeModule.java new file mode 100644 index 0000000000000000000000000000000000000000..fb8e2306222e1a7bda165e81be0c222f085ac003 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeModule.java @@ -0,0 +1,58 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.hetu.core.plugin.exchange.filesystem.hdfs.HdfsFileSystemExchangeStorage; +import io.hetu.core.plugin.exchange.filesystem.local.LocalFileSystemExchangeStorage; +import io.prestosql.spi.PrestoException; + +import java.net.URI; +import java.util.List; + +import static io.prestosql.spi.StandardErrorCode.CONFIGURATION_INVALID; +import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.String.format; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class FileSystemExchangeModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + binder.bind(FileSystemExchangeStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileSystemExchangeStats.class).withGeneratedName(); + + binder.bind(FileSystemExchangeManager.class).in(Scopes.SINGLETON); + + List baseDirectories = buildConfigObject(FileSystemExchangeConfig.class).getBaseDirectories(); + if (baseDirectories.stream().map(URI::getScheme).distinct().count() != 1) { + throw new PrestoException(CONFIGURATION_INVALID, "Multiple schemes in exchange base directories"); + } + String scheme = baseDirectories.get(0).getScheme(); + if (scheme == null || scheme.equals("file")) { + binder.bind(FileSystemExchangeStorage.class).to(LocalFileSystemExchangeStorage.class).in(Scopes.SINGLETON); + } + else if (ImmutableSet.of("hdfs").contains(scheme)) { + binder.bind(FileSystemExchangeStorage.class).to(HdfsFileSystemExchangeStorage.class).in(Scopes.SINGLETON); + } + else { + throw new PrestoException(NOT_SUPPORTED, format("Scheme %s is not supported as exchange storage", scheme)); + } + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangePlugin.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangePlugin.java new file mode 100644 index 0000000000000000000000000000000000000000..c93acd5f608f681df74411303ed8aeaa99ed5527 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangePlugin.java @@ -0,0 +1,28 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.exchange.ExchangeManagerFactory; + +public class FileSystemExchangePlugin + implements Plugin +{ + @Override + public Iterable getExchangeManagerFactories() + { + return ImmutableList.of(new FileSystemExchangeManagerFactory()); + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSink.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSink.java new file mode 100644 index 0000000000000000000000000000000000000000..9489b73ab876bac24817d1d61333e740917644a8 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSink.java @@ -0,0 +1,55 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.airlift.slice.Slice; +import io.prestosql.spi.exchange.ExchangeSink; + +import java.util.concurrent.CompletableFuture; + +public class FileSystemExchangeSink + implements ExchangeSink +{ + public static final String COMMITTED_MARKER_FILE_NAME = "committed"; + public static final String DATA_FILE_SUFFIX = ".data"; + + @Override + public CompletableFuture isBlocked() + { + return null; + } + + @Override + public void add(int partitionId, Slice slice) + { + } + + @Override + public long getMemoryUsage() + { + return 0; + } + + @Override + public CompletableFuture finish() + { + return null; + } + + @Override + public CompletableFuture abort() + { + return null; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkHandle.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..00eb2b258f7141fe8015ec18210cd7cfe3a47ce8 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkHandle.java @@ -0,0 +1,82 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.exchange.ExchangeSinkHandle; + +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class FileSystemExchangeSinkHandle + implements ExchangeSinkHandle +{ + private final int partitionId; + private final Optional secretKey; + + public FileSystemExchangeSinkHandle(@JsonProperty("partitionId") int partitionId, @JsonProperty("secretKey") Optional secretKey) + { + this.partitionId = partitionId; + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @JsonProperty + public Optional getSecretKey() + { + return secretKey; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FileSystemExchangeSinkHandle that = (FileSystemExchangeSinkHandle) o; + if (secretKey.isPresent() && that.secretKey.isPresent()) { + return partitionId == that.partitionId && Arrays.equals(secretKey.get(), that.secretKey.get()); + } + else { + return partitionId == that.partitionId && !secretKey.isPresent() && !that.secretKey.isPresent(); + } + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId, secretKey); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("partitionId", partitionId) + .add("secretKey", secretKey.map(val -> "[EDITED]")) + .toString(); + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkInstanceHandle.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkInstanceHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..603f7fb344352f979b3f65355d46577c4e5bb2a9 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSinkInstanceHandle.java @@ -0,0 +1,21 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.prestosql.spi.exchange.ExchangeSinkInstanceHandle; + +public class FileSystemExchangeSinkInstanceHandle + implements ExchangeSinkInstanceHandle +{ +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSourceHandle.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSourceHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..d4276ec67339fd9b429418ecada36d1307de831c --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeSourceHandle.java @@ -0,0 +1,41 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.exchange.ExchangeSourceHandle; +import org.openjdk.jol.info.ClassLayout; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class FileSystemExchangeSourceHandle + implements ExchangeSourceHandle +{ + private static final int INSTANCE_SIZE = ClassLayout.parseClass(FileSystemExchangeSourceHandle.class).instanceSize(); + + private final int partitionId; + private final Optional secretKey; + private List files; + + public FileSystemExchangeSourceHandle(@JsonProperty("partitionId") int partitionId, @JsonProperty("files") List files, @JsonProperty("secretKey") Optional secretKey) + { + this.partitionId = partitionId; + this.files = ImmutableList.copyOf(requireNonNull(files, "files is null")); + this.secretKey = requireNonNull(secretKey, "secretKey is null"); + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStats.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStats.java new file mode 100644 index 0000000000000000000000000000000000000000..b0bd83156b64341ed52d7444a24eb8ebdc9f37f3 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStats.java @@ -0,0 +1,86 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import io.airlift.stats.DistributionStat; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +public class FileSystemExchangeStats +{ + private final ExecutionStats createExchangeSourceHandles = new ExecutionStats(); + private final ExecutionStats getCommittedPartitions = new ExecutionStats(); + private final ExecutionStats closeExchange = new ExecutionStats(); + private final ExecutionStats exchangeSinkBlocked = new ExecutionStats(); + private final ExecutionStats exchangeSinkFinished = new ExecutionStats(); + private final ExecutionStats exchangeSinkAborted = new ExecutionStats(); + private final ExecutionStats exchangeSourceBlocked = new ExecutionStats(); + private final DistributionStat fileSizeInBytes = new DistributionStat(); + + @Managed + @Nested + public ExecutionStats getCreateExchangeSourceHandles() + { + return createExchangeSourceHandles; + } + + @Managed + @Nested + public ExecutionStats getGetCommittedPartitions() + { + return getCommittedPartitions; + } + + @Managed + @Nested + public ExecutionStats getCloseExchange() + { + return closeExchange; + } + + @Managed + @Nested + public ExecutionStats getExchangeSinkBlocked() + { + return exchangeSinkBlocked; + } + + @Managed + @Nested + public ExecutionStats getExchangeSinkFinished() + { + return exchangeSinkFinished; + } + + @Managed + @Nested + public ExecutionStats getExchangeSinkAborted() + { + return exchangeSinkAborted; + } + + @Managed + @Nested + public ExecutionStats getExchangeSourceBlocked() + { + return exchangeSourceBlocked; + } + + @Managed + @Nested + public DistributionStat getFileSizeInBytes() + { + return fileSizeInBytes; + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStorage.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStorage.java new file mode 100644 index 0000000000000000000000000000000000000000..2851fdbc9a35db433065d94e690a36bcae4cb08b --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/FileSystemExchangeStorage.java @@ -0,0 +1,42 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem; + +import com.google.common.util.concurrent.ListenableFuture; + +import javax.crypto.SecretKey; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Queue; + +public interface FileSystemExchangeStorage + extends AutoCloseable +{ + ExchangeReader createExchangeReader(Queue sourceFiles, int maxPageSize); + + ExchangeWriter createExchangeWriter(URI file, SecretKey secretKey); + + ListenableFuture createEmptyFile(URI file); + + ListenableFuture deleteRecursively(List directories); + + ListenableFuture> listFilesRecursively(URI dir); + + int getWriterBufferSize(); + + @Override + void close() throws IOException; +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/ExchangeHdfsConfig.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/ExchangeHdfsConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..b667bc3e3cb6f80da23aabec7c8d81091236e49d --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/ExchangeHdfsConfig.java @@ -0,0 +1,18 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem.hdfs; + +public class ExchangeHdfsConfig +{ +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/HdfsFileSystemExchangeStorage.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/HdfsFileSystemExchangeStorage.java new file mode 100644 index 0000000000000000000000000000000000000000..28b748177ebaec18ce6ed6c849539304d13beb3f --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/hdfs/HdfsFileSystemExchangeStorage.java @@ -0,0 +1,73 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem.hdfs; + +import com.google.common.util.concurrent.ListenableFuture; +import io.hetu.core.plugin.exchange.filesystem.ExchangeReader; +import io.hetu.core.plugin.exchange.filesystem.ExchangeSourceFile; +import io.hetu.core.plugin.exchange.filesystem.ExchangeWriter; +import io.hetu.core.plugin.exchange.filesystem.FileStatus; +import io.hetu.core.plugin.exchange.filesystem.FileSystemExchangeStorage; + +import javax.crypto.SecretKey; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Queue; + +public class HdfsFileSystemExchangeStorage + implements FileSystemExchangeStorage +{ + @Override + public ExchangeReader createExchangeReader(Queue sourceFiles, int maxPageSize) + { + return null; + } + + @Override + public ExchangeWriter createExchangeWriter(URI file, SecretKey secretKey) + { + return null; + } + + @Override + public ListenableFuture createEmptyFile(URI file) + { + return null; + } + + @Override + public ListenableFuture deleteRecursively(List directories) + { + return null; + } + + @Override + public ListenableFuture> listFilesRecursively(URI dir) + { + return null; + } + + @Override + public int getWriterBufferSize() + { + return 0; + } + + @Override + public void close() throws IOException + { + } +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/ExchangeLocalConfig.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/ExchangeLocalConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..9d7241c23e8c8bfc9acd85a46b134ac06e7abd28 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/ExchangeLocalConfig.java @@ -0,0 +1,18 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem.local; + +public class ExchangeLocalConfig +{ +} diff --git a/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java new file mode 100644 index 0000000000000000000000000000000000000000..814f329d687582bd5c688792578c17fb805d33e8 --- /dev/null +++ b/hetu-exchange-filesystem/src/main/java/io/hetu/core/plugin/exchange/filesystem/local/LocalFileSystemExchangeStorage.java @@ -0,0 +1,73 @@ +/* + * 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 io.hetu.core.plugin.exchange.filesystem.local; + +import com.google.common.util.concurrent.ListenableFuture; +import io.hetu.core.plugin.exchange.filesystem.ExchangeReader; +import io.hetu.core.plugin.exchange.filesystem.ExchangeSourceFile; +import io.hetu.core.plugin.exchange.filesystem.ExchangeWriter; +import io.hetu.core.plugin.exchange.filesystem.FileStatus; +import io.hetu.core.plugin.exchange.filesystem.FileSystemExchangeStorage; + +import javax.crypto.SecretKey; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Queue; + +public class LocalFileSystemExchangeStorage + implements FileSystemExchangeStorage +{ + @Override + public ExchangeReader createExchangeReader(Queue sourceFiles, int maxPageSize) + { + return null; + } + + @Override + public ExchangeWriter createExchangeWriter(URI file, SecretKey secretKey) + { + return null; + } + + @Override + public ListenableFuture createEmptyFile(URI file) + { + return null; + } + + @Override + public ListenableFuture deleteRecursively(List directories) + { + return null; + } + + @Override + public ListenableFuture> listFilesRecursively(URI dir) + { + return null; + } + + @Override + public int getWriterBufferSize() + { + return 0; + } + + @Override + public void close() throws IOException + { + } +} diff --git a/hetu-mpp/pom.xml b/hetu-mpp/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..673a13d212d4c7214203e2edc95f9ece21fcfd60 --- /dev/null +++ b/hetu-mpp/pom.xml @@ -0,0 +1,241 @@ + + +4.0.0 + + + io.hetu.core + presto-root + 1.8.0-SNAPSHOT + + +hetu-mpp +hetu - Mpp Connector +hetu-plugin + + + ${project.parent.basedir} + + + + + io.hetu.core + presto-hive + + + + io.airlift + json + + + + io.airlift + configuration + + + + io.airlift + bootstrap + + + + org.weakref + jmxutils + + + + io.airlift + event + + + + io.hetu.core + presto-plugin-toolkit + + + + io.airlift + log + + + + io.hetu.core + presto-orc + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.inject + javax.inject + + + + com.google.code.findbugs + jsr305 + true + + + + io.airlift + concurrent + + + + io.airlift + stats + + + + + io.hetu.core + presto-spi + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + net.jodah + expiringmap + 0.5.9 + + + + io.prestosql.hadoop + hadoop-apache + runtime + + + + commons-io + commons-io + runtime + + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + org.assertj + assertj-core + test + + + + io.hetu.core + presto-hive + test-jar + test + + + + io.prestosql.hive + hive-apache + + + + io.hetu.core + presto-main + test + + + + + + default + + true + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestHive.java + **/TestHiveFileSystemS3.java + **/TestHiveFileSystemS3SelectPushdown.java + + + + + + + + test-mpp + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestHive.java + + + + + + + + test-mpp-s3 + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestHiveFileSystemS3.java + **/TestHiveFileSystemS3SelectPushdown.java + + + + + + + + \ No newline at end of file diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConfig.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..dbf7bd4e631503b2385b3e7914df4a8a77ade413 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConfig.java @@ -0,0 +1,229 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp; + +import io.airlift.configuration.Config; + +public class MppConfig +{ + private String gdsList; + private boolean etlReuse; + private String baseAux; + private String auxUrl; + + private String hiveDb; + private String hiveUrl; + private String hiveUser; + private String hivePasswd; + private String hsqlDrop; + private String hsqlCreate; + + private String gsDriver; + private String gsUrl; + private String gsUser; + private String gsPasswd; + + private String gsqlCreate; + private String gsqlInsert; + private String gsqlDrop; + + public String getGdsList() + { + return gdsList; + } + + @Config("gds-list") + public MppConfig setGdsList(String gdsList) + { + this.gdsList = gdsList; + return this; + } + + public boolean isEtlReuse() + { + return etlReuse; + } + + @Config("etl-reuse") + public void setEtlReuse(boolean etlReuse) + { + this.etlReuse = etlReuse; + } + + public String getGsDriver() + { + return gsDriver; + } + + @Config("gs-driver") + public void setGsDriver(String gsDriver) + { + this.gsDriver = gsDriver; + } + + public String getGsUrl() + { + return gsUrl; + } + + @Config("gs-url") + public void setGsUrl(String gsUrl) + { + this.gsUrl = gsUrl; + } + + public String getGsUser() + { + return gsUser; + } + + @Config("gs-user") + public void setGsUser(String gsUser) + { + this.gsUser = gsUser; + } + + public String getGsPasswd() + { + return gsPasswd; + } + + @Config("gs-passwd") + public void setGsPasswd(String gsPasswd) + { + this.gsPasswd = gsPasswd; + } + + public String getGsqlCreate() + { + return gsqlCreate; + } + + @Config("gsql-create") + public void setGsqlCreate(String gsqlCreate) + { + this.gsqlCreate = gsqlCreate; + } + + public String getGsqlInsert() + { + return gsqlInsert; + } + + @Config("gsql-insert") + public void setGsqlInsert(String gsqlInsert) + { + this.gsqlInsert = gsqlInsert; + } + + public String getGsqlDrop() + { + return gsqlDrop; + } + + @Config("gsql-drop") + public void setGsqlDrop(String gsqlDrop) + { + this.gsqlDrop = gsqlDrop; + } + + public String getBaseAux() + { + return baseAux; + } + + @Config("base-aux") + public void setBaseAux(String baseAux) + { + this.baseAux = baseAux; + } + + public String getAuxUrl() + { + return auxUrl; + } + + @Config("aux-url") + public void setAuxUrl(String auxUrl) + { + this.auxUrl = auxUrl; + } + + public String getHiveDb() + { + return hiveDb; + } + + @Config("hive-db") + public void setHiveDb(String hiveDb) + { + this.hiveDb = hiveDb; + } + + public String getHiveUrl() + { + return hiveUrl; + } + + @Config("hive-url") + public void setHiveUrl(String hiveUrl) + { + this.hiveUrl = hiveUrl; + } + + public String getHiveUser() + { + return hiveUser; + } + + @Config("hive-user") + public void setHiveUser(String hiveUser) + { + this.hiveUser = hiveUser; + } + + public String getHivePasswd() + { + return hivePasswd; + } + + @Config("hive-passwd") + public void setHivePasswd(String hivePasswd) + { + this.hivePasswd = hivePasswd; + } + + public String getHsqlDrop() + { + return hsqlDrop; + } + + @Config("hsql-drop") + public void setHsqlDrop(String hsqlDrop) + { + this.hsqlDrop = hsqlDrop; + } + + public String getHsqlCreate() + { + return hsqlCreate; + } + + @Config("hsql-create") + public void setHsqlCreate(String hsqlCreate) + { + this.hsqlCreate = hsqlCreate; + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..6d143578543272a101a68619adc971f5b7c9c5e7 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java @@ -0,0 +1,154 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.TypeLiteral; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.event.client.EventModule; +import io.airlift.json.JsonModule; +import io.prestosql.plugin.base.jmx.MBeanServerModule; +import io.prestosql.plugin.hive.ConnectorObjectNameGeneratorModule; +import io.prestosql.plugin.hive.HiveAnalyzeProperties; +import io.prestosql.plugin.hive.HiveCatalogName; +import io.prestosql.plugin.hive.HiveConnector; +import io.prestosql.plugin.hive.HiveConnectorFactory; +import io.prestosql.plugin.hive.HiveMetadataFactory; +import io.prestosql.plugin.hive.HiveProcedureModule; +import io.prestosql.plugin.hive.HiveSchemaProperties; +import io.prestosql.plugin.hive.HiveSessionProperties; +import io.prestosql.plugin.hive.HiveTableProperties; +import io.prestosql.plugin.hive.HiveTransactionManager; +import io.prestosql.plugin.hive.NodeVersion; +import io.prestosql.plugin.hive.authentication.HiveAuthenticationModule; +import io.prestosql.plugin.hive.gcs.HiveGcsModule; +import io.prestosql.plugin.hive.metastore.HiveMetastore; +import io.prestosql.plugin.hive.metastore.HiveMetastoreModule; +import io.prestosql.plugin.hive.s3.HiveS3Module; +import io.prestosql.plugin.hive.security.HiveSecurityModule; +import io.prestosql.plugin.hive.security.SystemTableAwareAccessControl; +import io.prestosql.spi.NodeManager; +import io.prestosql.spi.PageIndexerFactory; +import io.prestosql.spi.PageSorter; +import io.prestosql.spi.VersionEmbedder; +import io.prestosql.spi.classloader.ThreadContextClassLoader; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorAccessControl; +import io.prestosql.spi.connector.ConnectorContext; +import io.prestosql.spi.connector.ConnectorNodePartitioningProvider; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSinkProvider; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeNodePartitioningProvider; +import io.prestosql.spi.heuristicindex.IndexClient; +import io.prestosql.spi.procedure.Procedure; +import io.prestosql.spi.type.TypeManager; +import org.weakref.jmx.guice.MBeanModule; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +public class MppConnectorFactory + extends HiveConnectorFactory +{ + private final ClassLoader classLoader; + + public MppConnectorFactory(String name, ClassLoader classLoader, Optional metastore) + { + super(name, classLoader, metastore); + this.classLoader = requireNonNull(classLoader, "classLoader is null"); + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + requireNonNull(config, "config is null"); + + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + Bootstrap app = new Bootstrap( + new EventModule(), + new MBeanModule(), + new ConnectorObjectNameGeneratorModule(catalogName), + new JsonModule(), + new MppModule(), + new HiveS3Module(), + new HiveGcsModule(), + new HiveMetastoreModule(Optional.empty()), + new HiveSecurityModule(), + new HiveAuthenticationModule(), + new HiveProcedureModule(), + new MBeanServerModule(), + binder -> { + binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(VersionEmbedder.class).toInstance(context.getVersionEmbedder()); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(PageIndexerFactory.class).toInstance(context.getPageIndexerFactory()); + binder.bind(PageSorter.class).toInstance(context.getPageSorter()); + binder.bind(HiveCatalogName.class).toInstance(new HiveCatalogName(catalogName)); + binder.bind(IndexClient.class).toInstance(context.getIndexClient()); + }); + + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + LifeCycleManager lifeCycleManager = injector.getInstance(LifeCycleManager.class); + HiveMetadataFactory metadataFactory = injector.getInstance(HiveMetadataFactory.class); + HiveTransactionManager transactionManager = injector.getInstance(HiveTransactionManager.class); + ConnectorSplitManager splitManager = injector.getInstance(ConnectorSplitManager.class); + ConnectorPageSourceProvider connectorPageSource = injector.getInstance(ConnectorPageSourceProvider.class); + ConnectorPageSinkProvider pageSinkProvider = injector.getInstance(ConnectorPageSinkProvider.class); + ConnectorNodePartitioningProvider connectorDistributionProvider = injector.getInstance(ConnectorNodePartitioningProvider.class); + HiveSessionProperties hiveSessionProperties = injector.getInstance(HiveSessionProperties.class); + HiveTableProperties hiveTableProperties = injector.getInstance(HiveTableProperties.class); + HiveAnalyzeProperties hiveAnalyzeProperties = injector.getInstance(HiveAnalyzeProperties.class); + ConnectorAccessControl accessControl = new SystemTableAwareAccessControl(injector.getInstance(ConnectorAccessControl.class)); + Set procedures = injector.getInstance(Key.get(new TypeLiteral>() {})); + + return new HiveConnector( + lifeCycleManager, + metadataFactory, + transactionManager, + new ClassLoaderSafeConnectorSplitManager(splitManager, classLoader), + new ClassLoaderSafeConnectorPageSourceProvider(connectorPageSource, classLoader), + new ClassLoaderSafeConnectorPageSinkProvider(pageSinkProvider, classLoader), + new ClassLoaderSafeNodePartitioningProvider(connectorDistributionProvider, classLoader), + ImmutableSet.of(), + procedures, + hiveSessionProperties.getSessionProperties(), + HiveSchemaProperties.SCHEMA_PROPERTIES, + hiveTableProperties.getTableProperties(), + hiveAnalyzeProperties.getAnalyzeProperties(), + accessControl, + classLoader); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadata.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadata.java new file mode 100644 index 0000000000000000000000000000000000000000..17f7ec1df90911e4b96407f015b3917f0941b22c --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadata.java @@ -0,0 +1,215 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import io.hetu.core.plugin.mpp.scheduler.Scheduler; +import io.hetu.core.plugin.mpp.scheduler.db.GsussDBOpt; +import io.hetu.core.plugin.mpp.scheduler.entity.ETLInfo; +import io.hetu.core.plugin.mpp.scheduler.utils.Const; +import io.hetu.core.plugin.mpp.scheduler.utils.Util; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HiveBucketing; +import io.prestosql.plugin.hive.HiveErrorCode; +import io.prestosql.plugin.hive.HiveMetadata; +import io.prestosql.plugin.hive.HivePartitionManager; +import io.prestosql.plugin.hive.HiveTableHandle; +import io.prestosql.plugin.hive.LocationService; +import io.prestosql.plugin.hive.PartitionUpdate; +import io.prestosql.plugin.hive.TypeTranslator; +import io.prestosql.plugin.hive.authentication.HiveIdentity; +import io.prestosql.plugin.hive.metastore.MetastoreUtil; +import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; +import io.prestosql.plugin.hive.metastore.Table; +import io.prestosql.plugin.hive.security.AccessControlMetadata; +import io.prestosql.plugin.hive.statistics.HiveStatisticsProvider; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.type.TypeManager; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; + +import static io.prestosql.plugin.hive.HiveUtil.getPartitionKeyColumnHandles; +import static java.util.Collections.emptyList; +import static java.util.Objects.requireNonNull; + +public class MppMetadata + extends HiveMetadata +{ + public Scheduler scheduler; + public MppConfig mppConfig; + public boolean createsOfNonManagedTablesEnabled; + public static Logger logger = Logger.get(MppMetadata.class); + + @Inject + public MppMetadata( + SemiTransactionalHiveMetastore metastore, + HdfsEnvironment hdfsEnvironment, + HivePartitionManager partitionManager, + boolean writesToNonManagedTablesEnabled, + boolean createsOfNonManagedTablesEnabled, + boolean tableCreatesWithLocationAllowed, + TypeManager typeManager, + LocationService locationService, + JsonCodec partitionUpdateCodec, + TypeTranslator typeTranslator, + String prestoVersion, + HiveStatisticsProvider hiveStatisticsProvider, + AccessControlMetadata accessControlMetadata, + boolean autoVacuumEnabled, + int vacuumDeltaNumThreshold, + double vacuumDeltaPercentThreshold, + ScheduledExecutorService vacuumExecutorService, + Optional vacuumCollectorInterval, + ScheduledExecutorService hiveMetastoreClientService, + Scheduler scheduler, + MppConfig mppConfig) + { + super(metastore, hdfsEnvironment, partitionManager, writesToNonManagedTablesEnabled, + createsOfNonManagedTablesEnabled, tableCreatesWithLocationAllowed, + typeManager, locationService, partitionUpdateCodec, typeTranslator, + prestoVersion, hiveStatisticsProvider, accessControlMetadata, + autoVacuumEnabled, vacuumDeltaNumThreshold, vacuumDeltaPercentThreshold, + vacuumExecutorService, vacuumCollectorInterval, hiveMetastoreClientService); + this.scheduler = scheduler; + this.mppConfig = mppConfig; + this.createsOfNonManagedTablesEnabled = createsOfNonManagedTablesEnabled; + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + List dbList = metastore.getAllDatabases(); + return dbList; + } + + @Override + public List listTables(ConnectorSession session, Optional optionalSchemaName) + { + ImmutableList.Builder tableNames = ImmutableList.builder(); + for (String schemaName : listSchemas(session, optionalSchemaName)) { + for (String tableName : metastore.getAllTables(schemaName).orElse(emptyList())) { + tableNames.add(new SchemaTableName(schemaName, tableName)); + } + } + return tableNames.build(); + } + + private List listSchemas(ConnectorSession session, Optional schemaName) + { + if (schemaName.isPresent()) { + return ImmutableList.of(schemaName.get()); + } + return listSchemaNames(session); + } + + @Override + public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + requireNonNull(tableName, "tableName is null"); + String gsSchemaName = tableName.getSchemaName(); + String schemaName = mppConfig.getHiveDb(); + String tblName = tableName.getTableName(); + String threadName = Const.tableStatus.getThreadName(); + Optional table = metastore.getTable(new HiveIdentity(session), schemaName, tableName.getTableName()); + String tblIdentifier = schemaName + "." + tblName; + logger.info("Mpp scheduler for " + tblIdentifier + " started"); + if (!table.isPresent()) { + logger.info("Hive(Mpp) table " + tblIdentifier + " is not present"); + determineWhetherToETL(gsSchemaName, schemaName, tblName, threadName, tblIdentifier); + } + else { + if (Const.etlInfoMap.containsKey(tblIdentifier) && Const.tableStatus.tableStatusKeysExists(tblIdentifier)) { +// have etled at least onece + if (mppConfig.isEtlReuse()) { + logger.info("Hive(Mpp) table " + tblIdentifier + " existed and reuse it"); + } + else { + logger.info("Hive(Mpp) " + tblIdentifier + " existed in this app runtime but not reuse it and will redo!"); + determineWhetherToETL(gsSchemaName, schemaName, tblName, threadName, tblIdentifier); + } + } + else { +// have etled in last restart, +// or have etled and still etling +// or just create and etling +// we can use two strategies to judge: rules and time interval + if (Const.tableStatus.tableStatusKeysExists(tblIdentifier)) { + logger.info("[2]Hive(MPP) " + tblIdentifier + " Table is existed and is etling by others and reuse it!"); + } + else { + logger.info("[2]Hive(MPP) " + tblIdentifier + " existed in last app runtime but not reuse it and will redo!"); + determineWhetherToETL(gsSchemaName, schemaName, tblName, threadName, tblIdentifier); + } + } + } + + table = metastore.getTable(new HiveIdentity(session), schemaName, tableName.getTableName()); + + // we must not allow system tables due to how permissions are checked in SystemTableAwareAccessControl + if (getSourceTableNameFromSystemTable(tableName).isPresent()) { + throw new PrestoException(HiveErrorCode.HIVE_INVALID_METADATA, "Unexpected table present in Hive metastore: " + tableName); + } + + MetastoreUtil.verifyOnline(tableName, Optional.empty(), MetastoreUtil.getProtectMode(table.get()), table.get().getParameters()); + + Map parameters = new HashMap<>(); + parameters.putAll(table.get().getParameters()); + + String format = table.get().getStorage().getStorageFormat().getOutputFormatNullable(); + if (format != null) { + parameters.put(STORAGE_FORMAT, format); + } + + return new HiveTableHandle( + schemaName, + tableName.getTableName(), + parameters, + getPartitionKeyColumnHandles(table.get()), + HiveBucketing.getHiveBucketHandle(table.get())); + } + + private void determineWhetherToETL(String gsSchemaName, String schemaName, String tblName, String threadName, String tblIdentifier) + { + List runningTaskList = Const.runningThreadMap.get(tblIdentifier); + Const.tableStatus.put(tblIdentifier, 0); + Const.etlInfoMap.put(tblIdentifier, new ETLInfo(0, Util.getDate(), "")); + int size; + String lock = TableMoveLock.getLock(tblIdentifier); + Map schemas = null; + synchronized (lock) { + runningTaskList.add(threadName); + Const.runningThreadMap.put(tblIdentifier, runningTaskList); + size = Const.runningThreadMap.get(tblIdentifier).size(); + if (size == 1) { + schemas = GsussDBOpt.getSchemas(mppConfig, "", gsSchemaName, tblName); + logger.info("Record table " + tblIdentifier + " into tableStatus and eltlInfoMap"); + scheduler.prepareHiveExternalTable(schemas, schemaName, tblName); + } + } + if (size == 1) { + Map.Entry gdsServer = scheduler.getGDS(); + scheduler.startGdsProcess(gdsServer, schemas, gsSchemaName, tblName); + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadataFactory.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadataFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..ae519024df4a54fae689bef51079ff03f27e4072 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppMetadataFactory.java @@ -0,0 +1,243 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import io.airlift.concurrent.BoundedExecutor; +import io.airlift.json.JsonCodec; +import io.airlift.units.Duration; +import io.hetu.core.plugin.mpp.scheduler.Scheduler; +import io.prestosql.plugin.hive.ForHive; +import io.prestosql.plugin.hive.ForHiveMetastore; +import io.prestosql.plugin.hive.ForHiveTransactionHeartbeats; +import io.prestosql.plugin.hive.ForHiveVacuum; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HiveConfig; +import io.prestosql.plugin.hive.HiveMetadata; +import io.prestosql.plugin.hive.HiveMetadataFactory; +import io.prestosql.plugin.hive.HivePartitionManager; +import io.prestosql.plugin.hive.LocationService; +import io.prestosql.plugin.hive.NodeVersion; +import io.prestosql.plugin.hive.PartitionUpdate; +import io.prestosql.plugin.hive.TypeTranslator; +import io.prestosql.plugin.hive.metastore.CachingHiveMetastore; +import io.prestosql.plugin.hive.metastore.HiveMetastore; +import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; +import io.prestosql.plugin.hive.security.AccessControlMetadataFactory; +import io.prestosql.plugin.hive.statistics.MetastoreHiveStatisticsProvider; +import io.prestosql.spi.type.TypeManager; + +import javax.inject.Inject; + +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; + +import static java.util.Objects.requireNonNull; + +public class MppMetadataFactory + extends HiveMetadataFactory +{ + private final boolean skipDeletionForAlter; + private final boolean skipTargetCleanupOnRollback; + private final boolean writesToNonManagedTablesEnabled; + private final boolean createsOfNonManagedTablesEnabled; + private final boolean tableCreatesWithLocationAllowed; + private final long perTransactionCacheMaximumSize; + private final HiveMetastore metastore; + private final HdfsEnvironment hdfsEnvironment; + private final HivePartitionManager partitionManager; + private final TypeManager typeManager; + private final LocationService locationService; + private final JsonCodec partitionUpdateCodec; + private final BoundedExecutor renameExecution; + private final ScheduledExecutorService hiveVacuumService; + private final TypeTranslator typeTranslator; + private final String prestoVersion; + private final AccessControlMetadataFactory accessControlMetadataFactory; + private final Optional hiveTransactionHeartbeatInterval; + private final ScheduledExecutorService heartbeatService; + private final ScheduledExecutorService hiveMetastoreClientService; + private final Duration vacuumCleanupRecheckInterval; + private final int vacuumDeltaNumThreshold; + private final double vacuumDeltaPercentThreshold; + private final boolean autoVacuumEnabled; + private Optional vacuumCollectorInterval; + protected final int hmsWriteBatchSize; + private Scheduler scheduler; + private MppConfig mppConfig; + SemiTransactionalHiveMetastore semiTransactionalHiveMetastore; + + @Inject + @SuppressWarnings("deprecation") + public MppMetadataFactory( + HiveConfig hiveConfig, + HiveMetastore metastore, + HdfsEnvironment hdfsEnvironment, + HivePartitionManager partitionManager, + @ForHive ExecutorService executorService, + @ForHiveVacuum ScheduledExecutorService hiveVacuumService, + @ForHiveMetastore ScheduledExecutorService hiveMetastoreClientService, + @ForHiveTransactionHeartbeats ScheduledExecutorService heartbeatService, + TypeManager typeManager, + LocationService locationService, + JsonCodec partitionUpdateCodec, + TypeTranslator typeTranslator, + NodeVersion nodeVersion, + AccessControlMetadataFactory accessControlMetadataFactory, + Scheduler scheduler, + MppConfig mppConfig) + { + this( + metastore, + hdfsEnvironment, + partitionManager, + hiveConfig.getMaxConcurrentFileRenames(), + hiveConfig.isSkipDeletionForAlter(), + hiveConfig.isSkipTargetCleanupOnRollback(), + hiveConfig.getWritesToNonManagedTablesEnabled(), + hiveConfig.getCreatesOfNonManagedTablesEnabled(), + hiveConfig.getTableCreatesWithLocationAllowed(), + hiveConfig.getPerTransactionMetastoreCacheMaximumSize(), + hiveConfig.getHiveTransactionHeartbeatInterval(), + hiveConfig.getVacuumCleanupRecheckInterval(), + typeManager, + locationService, + partitionUpdateCodec, + executorService, + hiveVacuumService, + heartbeatService, + hiveMetastoreClientService, + typeTranslator, + nodeVersion.toString(), + accessControlMetadataFactory, + hiveConfig.getVacuumDeltaNumThreshold(), + hiveConfig.getVacuumDeltaPercentThreshold(), + hiveConfig.getAutoVacuumEnabled(), + hiveConfig.getVacuumCollectorInterval(), + hiveConfig.getMetastoreWriteBatchSize(), + scheduler, + mppConfig); + } + + public MppMetadataFactory( + HiveMetastore metastore, + HdfsEnvironment hdfsEnvironment, + HivePartitionManager partitionManager, + int maxConcurrentFileRenames, + boolean skipDeletionForAlter, + boolean skipTargetCleanupOnRollback, + boolean writesToNonManagedTablesEnabled, + boolean createsOfNonManagedTablesEnabled, + boolean tableCreatesWithLocationAllowed, + long perTransactionCacheMaximumSize, + Optional hiveTransactionHeartbeatInterval, + Duration vacuumCleanupRecheckInterval, + TypeManager typeManager, + LocationService locationService, + JsonCodec partitionUpdateCodec, + ExecutorService executorService, + ScheduledExecutorService hiveVacuumService, + ScheduledExecutorService heartbeatService, + ScheduledExecutorService hiveMetastoreClientService, + TypeTranslator typeTranslator, + String prestoVersion, + AccessControlMetadataFactory accessControlMetadataFactory, + int vacuumDeltaNumThreshold, + double vacuumDeltaPercentThreshold, + boolean autoVacuumEnabled, + Optional vacuumCollectorInterval, + int hmsWriteBatchSize, + Scheduler scheduler, + MppConfig mppConfig) + { + super(metastore, hdfsEnvironment, partitionManager, maxConcurrentFileRenames, + skipDeletionForAlter, skipTargetCleanupOnRollback, writesToNonManagedTablesEnabled, + createsOfNonManagedTablesEnabled, tableCreatesWithLocationAllowed, perTransactionCacheMaximumSize, + hiveTransactionHeartbeatInterval, vacuumCleanupRecheckInterval, typeManager, locationService, + partitionUpdateCodec, executorService, hiveVacuumService, heartbeatService, + hiveMetastoreClientService, typeTranslator, prestoVersion, accessControlMetadataFactory, + vacuumDeltaNumThreshold, vacuumDeltaPercentThreshold, autoVacuumEnabled, vacuumCollectorInterval, hmsWriteBatchSize); + + this.skipDeletionForAlter = skipDeletionForAlter; + this.skipTargetCleanupOnRollback = skipTargetCleanupOnRollback; + this.writesToNonManagedTablesEnabled = writesToNonManagedTablesEnabled; + this.createsOfNonManagedTablesEnabled = createsOfNonManagedTablesEnabled; + this.tableCreatesWithLocationAllowed = tableCreatesWithLocationAllowed; + this.perTransactionCacheMaximumSize = perTransactionCacheMaximumSize; + + this.metastore = requireNonNull(metastore, "metastore is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.partitionManager = requireNonNull(partitionManager, "partitionManager is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.locationService = requireNonNull(locationService, "locationService is null"); + this.partitionUpdateCodec = requireNonNull(partitionUpdateCodec, "partitionUpdateCodec is null"); + this.typeTranslator = requireNonNull(typeTranslator, "typeTranslator is null"); + this.prestoVersion = requireNonNull(prestoVersion, "prestoVersion is null"); + this.accessControlMetadataFactory = requireNonNull(accessControlMetadataFactory, "accessControlMetadataFactory is null"); + this.hiveTransactionHeartbeatInterval = requireNonNull(hiveTransactionHeartbeatInterval, "hiveTransactionHeartbeatInterval is null"); + this.vacuumCleanupRecheckInterval = requireNonNull(vacuumCleanupRecheckInterval, "vacuumCleanupInterval is null"); + + renameExecution = new BoundedExecutor(executorService, maxConcurrentFileRenames); + this.hiveVacuumService = requireNonNull(hiveVacuumService, "hiveVacuumService is null"); + this.heartbeatService = requireNonNull(heartbeatService, "heartbeatService is null"); + this.hiveMetastoreClientService = requireNonNull(hiveMetastoreClientService, "heartbeatService is null"); + this.vacuumDeltaNumThreshold = vacuumDeltaNumThreshold; + this.vacuumDeltaPercentThreshold = vacuumDeltaPercentThreshold; + this.autoVacuumEnabled = autoVacuumEnabled; + this.vacuumCollectorInterval = vacuumCollectorInterval; + this.hmsWriteBatchSize = hmsWriteBatchSize; + this.scheduler = scheduler; + this.mppConfig = mppConfig; + } + + @Override + public HiveMetadata get() + { + semiTransactionalHiveMetastore = new SemiTransactionalHiveMetastore( + hdfsEnvironment, + CachingHiveMetastore.memoizeMetastore(this.metastore, perTransactionCacheMaximumSize), // per-transaction cache + renameExecution, + hiveVacuumService, + vacuumCleanupRecheckInterval, + skipDeletionForAlter, + skipTargetCleanupOnRollback, + hiveTransactionHeartbeatInterval, + heartbeatService, + hiveMetastoreClientService, + hmsWriteBatchSize); + + return new MppMetadata( + semiTransactionalHiveMetastore, + hdfsEnvironment, + partitionManager, + writesToNonManagedTablesEnabled, + createsOfNonManagedTablesEnabled, + tableCreatesWithLocationAllowed, + typeManager, + locationService, + partitionUpdateCodec, + typeTranslator, + prestoVersion, + new MetastoreHiveStatisticsProvider(semiTransactionalHiveMetastore, statsCache, samplePartitionCache), + accessControlMetadataFactory.create(semiTransactionalHiveMetastore), + autoVacuumEnabled, + vacuumDeltaNumThreshold, + vacuumDeltaPercentThreshold, + hiveVacuumService, + vacuumCollectorInterval, + hiveMetastoreClientService, + scheduler, + mppConfig); + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppModule.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppModule.java new file mode 100644 index 0000000000000000000000000000000000000000..40f7b5870c7f84c1deab686cc17ba83bc2b8d60f --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppModule.java @@ -0,0 +1,173 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import com.google.common.cache.CacheLoader; +import com.google.inject.Binder; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.Multibinder; +import io.airlift.event.client.EventClient; +import io.hetu.core.plugin.mpp.scheduler.Scheduler; +import io.prestosql.orc.BloomFilterCacheStatsLister; +import io.prestosql.orc.FileTailCacheStatsLister; +import io.prestosql.orc.RowDataCacheStatsLister; +import io.prestosql.orc.RowIndexCacheStatsLister; +import io.prestosql.orc.StripeFooterCacheStatsLister; +import io.prestosql.plugin.hive.CachingDirectoryLister; +import io.prestosql.plugin.hive.CoercionPolicy; +import io.prestosql.plugin.hive.DirectoryLister; +import io.prestosql.plugin.hive.DynamicConfigurationProvider; +import io.prestosql.plugin.hive.FileFormatDataSourceStats; +import io.prestosql.plugin.hive.GenericHiveRecordCursorProvider; +import io.prestosql.plugin.hive.HdfsConfiguration; +import io.prestosql.plugin.hive.HdfsConfigurationInitializer; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HiveAnalyzeProperties; +import io.prestosql.plugin.hive.HiveCoercionPolicy; +import io.prestosql.plugin.hive.HiveConfig; +import io.prestosql.plugin.hive.HiveEventClient; +import io.prestosql.plugin.hive.HiveFileWriterFactory; +import io.prestosql.plugin.hive.HiveHdfsConfiguration; +import io.prestosql.plugin.hive.HiveLocationService; +import io.prestosql.plugin.hive.HiveMetadataFactory; +import io.prestosql.plugin.hive.HiveModule; +import io.prestosql.plugin.hive.HiveNodePartitioningProvider; +import io.prestosql.plugin.hive.HivePageSinkProvider; +import io.prestosql.plugin.hive.HivePageSourceFactory; +import io.prestosql.plugin.hive.HivePageSourceProvider; +import io.prestosql.plugin.hive.HivePartitionManager; +import io.prestosql.plugin.hive.HiveRecordCursorProvider; +import io.prestosql.plugin.hive.HiveSelectivePageSourceFactory; +import io.prestosql.plugin.hive.HiveSessionProperties; +import io.prestosql.plugin.hive.HiveTableProperties; +import io.prestosql.plugin.hive.HiveTransactionManager; +import io.prestosql.plugin.hive.HiveTypeTranslator; +import io.prestosql.plugin.hive.HiveWriterStats; +import io.prestosql.plugin.hive.LocationService; +import io.prestosql.plugin.hive.NamenodeStats; +import io.prestosql.plugin.hive.OrcFileWriterConfig; +import io.prestosql.plugin.hive.OrcFileWriterFactory; +import io.prestosql.plugin.hive.ParquetFileWriterConfig; +import io.prestosql.plugin.hive.PartitionUpdate; +import io.prestosql.plugin.hive.RcFileFileWriterFactory; +import io.prestosql.plugin.hive.S3SelectRecordCursorProvider; +import io.prestosql.plugin.hive.TransactionalMetadata; +import io.prestosql.plugin.hive.TypeTranslator; +import io.prestosql.plugin.hive.orc.OrcPageSourceFactory; +import io.prestosql.plugin.hive.orc.OrcSelectivePageSourceFactory; +import io.prestosql.plugin.hive.parquet.ParquetPageSourceFactory; +import io.prestosql.plugin.hive.rcfile.RcFilePageSourceFactory; +import io.prestosql.plugin.hive.s3.PrestoS3ClientFactory; +import io.prestosql.plugin.hive.util.IndexCache; +import io.prestosql.plugin.hive.util.IndexCacheLoader; +import io.prestosql.spi.connector.ConnectorNodePartitioningProvider; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorPageSourceProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; + +import java.util.function.Supplier; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class MppModule + extends HiveModule +{ + @Override + public void configure(Binder binder) + { + binder.bind(TypeTranslator.class).toInstance(new HiveTypeTranslator()); + binder.bind(CoercionPolicy.class).to(HiveCoercionPolicy.class).in(Scopes.SINGLETON); + + binder.bind(HdfsConfigurationInitializer.class).in(Scopes.SINGLETON); + newSetBinder(binder, DynamicConfigurationProvider.class); + binder.bind(HdfsConfiguration.class).to(HiveHdfsConfiguration.class).in(Scopes.SINGLETON); + binder.bind(HdfsEnvironment.class).in(Scopes.SINGLETON); + binder.bind(DirectoryLister.class).to(CachingDirectoryLister.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(HiveConfig.class); + + binder.bind(HiveSessionProperties.class).in(Scopes.SINGLETON); + binder.bind(HiveTableProperties.class).in(Scopes.SINGLETON); + binder.bind(HiveAnalyzeProperties.class).in(Scopes.SINGLETON); + + binder.bind(NamenodeStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(NamenodeStats.class).withGeneratedName(); + + binder.bind(PrestoS3ClientFactory.class).in(Scopes.SINGLETON); + + binder.bind(CachingDirectoryLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(CachingDirectoryLister.class).withGeneratedName(); + + Multibinder recordCursorProviderBinder = newSetBinder(binder, HiveRecordCursorProvider.class); + recordCursorProviderBinder.addBinding().to(S3SelectRecordCursorProvider.class).in(Scopes.SINGLETON); + recordCursorProviderBinder.addBinding().to(GenericHiveRecordCursorProvider.class).in(Scopes.SINGLETON); + + binder.bind(HiveWriterStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(HiveWriterStats.class).withGeneratedName(); + + newSetBinder(binder, EventClient.class).addBinding().to(HiveEventClient.class).in(Scopes.SINGLETON); + binder.bind(HivePartitionManager.class).in(Scopes.SINGLETON); + binder.bind(LocationService.class).to(HiveLocationService.class).in(Scopes.SINGLETON); + binder.bind(HiveMetadataFactory.class).to(MppMetadataFactory.class).in(Scopes.SINGLETON); + binder.bind(new TypeLiteral>() {}).to(HiveMetadataFactory.class).in(Scopes.SINGLETON); + binder.bind(HiveTransactionManager.class).in(Scopes.SINGLETON); + binder.bind(ConnectorSplitManager.class).to(MppSplitManager.class).in(Scopes.SINGLETON); + newExporter(binder).export(ConnectorSplitManager.class).as(generator -> generator.generatedNameOf(MppSplitManager.class)); + binder.bind(ConnectorPageSourceProvider.class).to(HivePageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorPageSinkProvider.class).to(HivePageSinkProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorNodePartitioningProvider.class).to(HiveNodePartitioningProvider.class).in(Scopes.SINGLETON); + + jsonCodecBinder(binder).bindJsonCodec(PartitionUpdate.class); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); + + Multibinder pageSourceFactoryBinder = newSetBinder(binder, HivePageSourceFactory.class); + pageSourceFactoryBinder.addBinding().to(OrcPageSourceFactory.class).in(Scopes.SINGLETON); + pageSourceFactoryBinder.addBinding().to(ParquetPageSourceFactory.class).in(Scopes.SINGLETON); + pageSourceFactoryBinder.addBinding().to(RcFilePageSourceFactory.class).in(Scopes.SINGLETON); + + Multibinder selectivePageSourceFactoryBinder = newSetBinder(binder, HiveSelectivePageSourceFactory.class); + selectivePageSourceFactoryBinder.addBinding().to(OrcSelectivePageSourceFactory.class).in(Scopes.SINGLETON); + + Multibinder fileWriterFactoryBinder = newSetBinder(binder, HiveFileWriterFactory.class); + binder.bind(OrcFileWriterFactory.class).in(Scopes.SINGLETON); + newExporter(binder).export(OrcFileWriterFactory.class).withGeneratedName(); + configBinder(binder).bindConfig(OrcFileWriterConfig.class); + fileWriterFactoryBinder.addBinding().to(OrcFileWriterFactory.class).in(Scopes.SINGLETON); + fileWriterFactoryBinder.addBinding().to(RcFileFileWriterFactory.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(ParquetFileWriterConfig.class); + + binder.bind(CacheLoader.class).to(IndexCacheLoader.class).in(Scopes.SINGLETON); + binder.bind(IndexCache.class).in(Scopes.SINGLETON); + + binder.bind(FileTailCacheStatsLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileTailCacheStatsLister.class).withGeneratedName(); + binder.bind(StripeFooterCacheStatsLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(StripeFooterCacheStatsLister.class).withGeneratedName(); + binder.bind(RowIndexCacheStatsLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(RowIndexCacheStatsLister.class).withGeneratedName(); + binder.bind(BloomFilterCacheStatsLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(BloomFilterCacheStatsLister.class).withGeneratedName(); + binder.bind(RowDataCacheStatsLister.class).in(Scopes.SINGLETON); + newExporter(binder).export(RowDataCacheStatsLister.class).withGeneratedName(); + + configBinder(binder).bindConfig(MppConfig.class); + binder.bind(Scheduler.class).in(Scopes.SINGLETON); + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppPlugin.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppPlugin.java new file mode 100644 index 0000000000000000000000000000000000000000..4d89d136849eb30017b8a9dabe535d44bb19eccc --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppPlugin.java @@ -0,0 +1,35 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import com.google.common.collect.ImmutableList; +import io.prestosql.plugin.hive.HivePlugin; +import io.prestosql.spi.connector.ConnectorFactory; + +import java.util.Optional; + +public class MppPlugin + extends HivePlugin +{ + public MppPlugin() + { + super("mpp"); + } + + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new MppConnectorFactory("mpp", MppPlugin.class.getClassLoader(), Optional.empty())); + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppSplitManager.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppSplitManager.java new file mode 100644 index 0000000000000000000000000000000000000000..4e5dcd9561c4b50e6aa601e98431520455abe8ae --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppSplitManager.java @@ -0,0 +1,361 @@ +/* + * 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 io.hetu.core.plugin.mpp; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; +import io.airlift.concurrent.BoundedExecutor; +import io.airlift.log.Logger; +import io.airlift.stats.CounterStat; +import io.airlift.units.DataSize; +import io.hetu.core.plugin.mpp.scheduler.entity.ETLInfo; +import io.hetu.core.plugin.mpp.scheduler.utils.Const; +import io.hetu.core.plugin.mpp.scheduler.utils.Util; +import io.prestosql.plugin.hive.BackgroundHiveSplitLoader; +import io.prestosql.plugin.hive.CoercionPolicy; +import io.prestosql.plugin.hive.DirectoryLister; +import io.prestosql.plugin.hive.ForHive; +import io.prestosql.plugin.hive.HdfsEnvironment; +import io.prestosql.plugin.hive.HiveBucketHandle; +import io.prestosql.plugin.hive.HiveBucketing; +import io.prestosql.plugin.hive.HiveConfig; +import io.prestosql.plugin.hive.HiveMetadata; +import io.prestosql.plugin.hive.HiveNotReadableException; +import io.prestosql.plugin.hive.HivePartition; +import io.prestosql.plugin.hive.HivePartitionManager; +import io.prestosql.plugin.hive.HivePartitionMetadata; +import io.prestosql.plugin.hive.HiveSplitLoader; +import io.prestosql.plugin.hive.HiveSplitManager; +import io.prestosql.plugin.hive.HiveSplitSource; +import io.prestosql.plugin.hive.HiveStorageFormat; +import io.prestosql.plugin.hive.HiveTableHandle; +import io.prestosql.plugin.hive.HiveTransactionHandle; +import io.prestosql.plugin.hive.HiveVacuumSplitSource; +import io.prestosql.plugin.hive.HiveVacuumTableHandle; +import io.prestosql.plugin.hive.NamenodeStats; +import io.prestosql.plugin.hive.authentication.HiveIdentity; +import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore; +import io.prestosql.plugin.hive.metastore.Table; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.VersionEmbedder; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplitSource; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.FixedSplitSource; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.TableNotFoundException; +import io.prestosql.spi.dynamicfilter.DynamicFilter; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.resourcegroups.QueryType; +import io.prestosql.spi.type.TypeManager; + +import javax.annotation.Nullable; +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; +import java.util.function.Function; +import java.util.function.Supplier; + +import static com.google.common.base.MoreObjects.firstNonNull; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.prestosql.spi.StandardErrorCode.SERVER_SHUTTING_DOWN; +import static io.prestosql.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING; +import static java.util.Objects.requireNonNull; + +public class MppSplitManager + extends HiveSplitManager +{ + public static Logger logger = Logger.get(MppSplitManager.class); + public static final String OBJECT_NOT_READABLE = "object_not_readable"; + + private final Function metastoreProvider; + private final HivePartitionManager partitionManager; + private final NamenodeStats namenodeStats; + private final HdfsEnvironment hdfsEnvironment; + private final DirectoryLister directoryLister; + private final Executor executor; + private final CoercionPolicy coercionPolicy; + private final int maxOutstandingSplits; + private final DataSize maxOutstandingSplitsSize; + private final int minPartitionBatchSize; + private final int maxPartitionBatchSize; + private final int maxInitialSplits; + private final int splitLoaderConcurrency; + private final int maxSplitsPerSecond; + private final boolean recursiveDfsWalkerEnabled; + private final CounterStat highMemorySplitSourceCounter; + private final TypeManager typeManager; + private final HiveConfig hiveConfig; + + @Inject + public MppSplitManager( + HiveConfig hiveConfig, + Function metastoreProvider, + HivePartitionManager partitionManager, + NamenodeStats namenodeStats, + HdfsEnvironment hdfsEnvironment, + DirectoryLister directoryLister, + @ForHive ExecutorService executorService, + VersionEmbedder versionEmbedder, + TypeManager typeManager, + CoercionPolicy coercionPolicy) + { + this( + metastoreProvider, + partitionManager, + namenodeStats, + hdfsEnvironment, + directoryLister, + versionEmbedder.embedVersion(new BoundedExecutor(executorService, hiveConfig.getMaxSplitIteratorThreads())), + coercionPolicy, + new CounterStat(), + hiveConfig.getMaxOutstandingSplits(), + hiveConfig.getMaxOutstandingSplitsSize(), + hiveConfig.getMinPartitionBatchSize(), + hiveConfig.getMaxPartitionBatchSize(), + hiveConfig.getMaxInitialSplits(), + hiveConfig.getSplitLoaderConcurrency(), + hiveConfig.getMaxSplitsPerSecond(), + hiveConfig.getRecursiveDirWalkerEnabled(), + typeManager, + hiveConfig); + } + + public MppSplitManager( + Function metastoreProvider, + HivePartitionManager partitionManager, + NamenodeStats namenodeStats, + HdfsEnvironment hdfsEnvironment, + DirectoryLister directoryLister, + Executor executor, + CoercionPolicy coercionPolicy, + CounterStat highMemorySplitSourceCounter, + int maxOutstandingSplits, + DataSize maxOutstandingSplitsSize, + int minPartitionBatchSize, + int maxPartitionBatchSize, + int maxInitialSplits, + int splitLoaderConcurrency, + @Nullable Integer maxSplitsPerSecond, + boolean recursiveDfsWalkerEnabled, + TypeManager typeManager, + HiveConfig hiveConfig) + { + super(metastoreProvider, partitionManager, namenodeStats, + hdfsEnvironment, directoryLister, executor, coercionPolicy, + highMemorySplitSourceCounter, maxOutstandingSplits, + maxOutstandingSplitsSize, minPartitionBatchSize, + maxPartitionBatchSize, maxInitialSplits, splitLoaderConcurrency, + maxSplitsPerSecond, recursiveDfsWalkerEnabled, typeManager, hiveConfig); + + this.metastoreProvider = requireNonNull(metastoreProvider, "metastore is null"); + this.partitionManager = requireNonNull(partitionManager, "partitionManager is null"); + this.namenodeStats = requireNonNull(namenodeStats, "namenodeStats is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.directoryLister = requireNonNull(directoryLister, "directoryLister is null"); + this.executor = new ErrorCodedExecutor(executor); + this.coercionPolicy = requireNonNull(coercionPolicy, "coercionPolicy is null"); + this.highMemorySplitSourceCounter = requireNonNull(highMemorySplitSourceCounter, "highMemorySplitSourceCounter is null"); + checkArgument(maxOutstandingSplits >= 1, "maxOutstandingSplits must be at least 1"); + this.maxOutstandingSplits = maxOutstandingSplits; + this.maxOutstandingSplitsSize = maxOutstandingSplitsSize; + this.minPartitionBatchSize = minPartitionBatchSize; + this.maxPartitionBatchSize = maxPartitionBatchSize; + this.maxInitialSplits = maxInitialSplits; + this.splitLoaderConcurrency = splitLoaderConcurrency; + this.maxSplitsPerSecond = firstNonNull(maxSplitsPerSecond, Integer.MAX_VALUE); + this.recursiveDfsWalkerEnabled = recursiveDfsWalkerEnabled; + this.typeManager = typeManager; + this.hiveConfig = hiveConfig; + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle tableHandle, + SplitSchedulingStrategy splitSchedulingStrategy, + Supplier>> dynamicFilterSupplier, + Optional queryType, + Map queryInfo, + Set> userDefinedCachePredicates, + boolean partOfReuse) + { + HiveTableHandle hiveTable = (HiveTableHandle) tableHandle; + SchemaTableName tableName = hiveTable.getSchemaTableName(); + + String tblIdentifier = tableName.getSchemaName() + "." + tableName.getTableName(); + Integer recode = Const.tableStatus.get(tblIdentifier) != null ? Const.tableStatus.get(tblIdentifier) : -1; + + if (recode == -1) { + logger.info("Have not find the gaussdb table's status info, maybe this is a reuse scheduler!"); + } + else { + logger.info("Find the gaussdb table's etl status info!"); + while (recode != 1) { + try { + Thread.sleep(2000); + logger.info("Waitting to complete GDS process transporting data to alluxio"); + } + catch (InterruptedException e) { + logger.error(e.getMessage()); + } + recode = Const.tableStatus.get(tblIdentifier); + } + + Const.tableStatus.remove(tblIdentifier); + ETLInfo etlInfo = Const.etlInfoMap.get(tblIdentifier); + Const.etlInfoMap.put(tblIdentifier, new ETLInfo(1, etlInfo.getStartTime(), Util.getDate())); + } + + // get table metadata + SemiTransactionalHiveMetastore metastore = metastoreProvider.apply((HiveTransactionHandle) transaction); + Table table = metastore.getTable(new HiveIdentity(session), tableName.getSchemaName(), tableName.getTableName()) + .orElseThrow(() -> new TableNotFoundException(tableName)); + if (table.getStorage().getStorageFormat().getInputFormat().contains("carbon")) { + throw new PrestoException(NOT_SUPPORTED, "Hive connector can't read carbondata tables"); + } + + // verify table is not marked as non-readable + String tableNotReadable = table.getParameters().get(OBJECT_NOT_READABLE); + if (!isNullOrEmpty(tableNotReadable)) { + throw new HiveNotReadableException(tableName, Optional.empty(), tableNotReadable); + } + + // get partitions + List partitions = partitionManager.getOrLoadPartitions(session, metastore, new HiveIdentity(session), hiveTable); + + // short circuit if we don't have any partitions + if (partitions.isEmpty()) { + return new FixedSplitSource(ImmutableList.of()); + } + + // get buckets from first partition (arbitrary) + Optional bucketFilter = hiveTable.getBucketFilter(); + + // validate bucket bucketed execution + Optional bucketHandle = hiveTable.getBucketHandle(); + if ((splitSchedulingStrategy == GROUPED_SCHEDULING) && !bucketHandle.isPresent()) { + throw new PrestoException(GENERIC_INTERNAL_ERROR, "SchedulingPolicy is bucketed, but BucketHandle is not present"); + } + + // sort partitions + partitions = Ordering.natural().onResultOf(HivePartition::getPartitionId).reverse().sortedCopy(partitions); + + Iterable hivePartitions = getPartitionMetadata(session, metastore, table, tableName, partitions, bucketHandle.map(HiveBucketHandle::toTableBucketProperty)); + + HiveSplitLoader hiveSplitLoader = new BackgroundHiveSplitLoader( + table, + hivePartitions, + hiveTable.getCompactEffectivePredicate(), + BackgroundHiveSplitLoader.BucketSplitInfo.createBucketSplitInfo(bucketHandle, bucketFilter), + session, + hdfsEnvironment, + namenodeStats, + directoryLister, + executor, + splitLoaderConcurrency, + recursiveDfsWalkerEnabled, + metastore.getValidWriteIds(session, hiveTable, queryType.map(t -> t == QueryType.VACUUM).orElse(false)) + .map(validTxnWriteIdList -> validTxnWriteIdList.getTableValidWriteIdList(table.getDatabaseName() + "." + table.getTableName())), + dynamicFilterSupplier, + queryType, + queryInfo, + typeManager); + + HiveSplitSource splitSource; + HiveStorageFormat hiveStorageFormat = HiveMetadata.extractHiveStorageFormat(table); + switch (splitSchedulingStrategy) { + case UNGROUPED_SCHEDULING: + splitSource = HiveSplitSource.allAtOnce( + session, + table.getDatabaseName(), + table.getTableName(), + partOfReuse ? 0 : maxInitialSplits, //For reuse, we should make sure to have same split size all time for a table. + maxOutstandingSplits, + maxOutstandingSplitsSize, + maxSplitsPerSecond, + hiveSplitLoader, + executor, + new CounterStat(), + dynamicFilterSupplier, + userDefinedCachePredicates, + typeManager, + hiveConfig, + hiveStorageFormat); + break; + case GROUPED_SCHEDULING: + splitSource = HiveSplitSource.bucketed( + session, + table.getDatabaseName(), + table.getTableName(), + partOfReuse ? 0 : maxInitialSplits, //For reuse, we should make sure to have same split size all time for a table. + maxOutstandingSplits, + maxOutstandingSplitsSize, + maxSplitsPerSecond, + hiveSplitLoader, + executor, + new CounterStat(), + dynamicFilterSupplier, + userDefinedCachePredicates, + typeManager, + hiveConfig, + hiveStorageFormat); + break; + default: + throw new IllegalArgumentException("Unknown splitSchedulingStrategy: " + splitSchedulingStrategy); + } + hiveSplitLoader.start(splitSource); + + if (queryType.isPresent() && queryType.get() == QueryType.VACUUM) { + HdfsEnvironment.HdfsContext hdfsContext = new HdfsEnvironment.HdfsContext(session, table.getDatabaseName(), table.getTableName()); + return new HiveVacuumSplitSource(splitSource, (HiveVacuumTableHandle) queryInfo.get("vacuumHandle"), hdfsEnvironment, hdfsContext, session); + } + + return splitSource; + } + + private static class ErrorCodedExecutor + implements Executor + { + private final Executor delegate; + + private ErrorCodedExecutor(Executor delegate) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + } + + @Override + public void execute(Runnable command) + { + try { + delegate.execute(command); + } + catch (RejectedExecutionException e) { + throw new PrestoException(SERVER_SHUTTING_DOWN, "Server is shutting down", e); + } + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/RunningTaskHashMap.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/RunningTaskHashMap.java new file mode 100644 index 0000000000000000000000000000000000000000..c2188d8853d3e5bfeb4a01b4693da77080b20aed --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/RunningTaskHashMap.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp; + +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + +public class RunningTaskHashMap +{ + public HashMap> hashMap; + + public RunningTaskHashMap() + { + this.hashMap = new HashMap<>(); + } + + public List put(String key, List value) + { + synchronized (this) { + hashMap.put(key, value); + return value; + } + } + + public List get(String key) + { + synchronized (this) { + if (hashMap.get(key) == null) { + List list = new CopyOnWriteArrayList<>(); + hashMap.put(key, list); + return hashMap.get(key); + } + else { + return hashMap.get(key); + } + } + } + + public List removeList(Object key) + { + synchronized (this) { + return hashMap.remove(key); + } + } + + public boolean removeThread(Object key, String threadName) + { + synchronized (this) { + if (hashMap.get(key) != null) { + return hashMap.get(key).remove(threadName); + } + else { + return true; + } + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/SynchronizedHashMap.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/SynchronizedHashMap.java new file mode 100644 index 0000000000000000000000000000000000000000..9e437c1c6acdbc635f2548cefe8bae1abdb27a8e --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/SynchronizedHashMap.java @@ -0,0 +1,258 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp; + +import io.hetu.core.plugin.mpp.scheduler.utils.Const; +import net.jodah.expiringmap.ExpirationPolicy; +import net.jodah.expiringmap.ExpiringMap; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class SynchronizedHashMap +{ + public HashMap hashMap; + public String mapName; + public static final String TABLE_STATUS_MAP = "tableStatus"; + public static final String ETL_INFO_MAP = "etlInfo"; + + public static final String TABLE_SCHEMA_MAP = "schemas"; + + public HashMap> threadHashMap; + + public List tableStatusList; + + public ExpiringMap expiringMap; + + public HashMap gaussDbTaskStatusMap; + + public int second; + + private SynchronizedHashMap() + { + } + + public SynchronizedHashMap(String mapName) + { + this.mapName = mapName; + switch (mapName){ + case TABLE_STATUS_MAP: + threadHashMap = new HashMap(); + tableStatusList = new ArrayList<>(); + gaussDbTaskStatusMap = new HashMap<>(); + break; + case ETL_INFO_MAP: + hashMap = new HashMap(); + break; + default: + break; + } + } + + public SynchronizedHashMap(int second, int maxSize) + { + this.second = second; + this.mapName = TABLE_SCHEMA_MAP; + expiringMap = ExpiringMap.builder() + .maxSize(maxSize) + .expirationPolicy(ExpirationPolicy.ACCESSED) + .expiration(second, TimeUnit.SECONDS) + .build(); + } + + public V put(K key, V value, String threadName) + { + switch (mapName){ + case TABLE_STATUS_MAP: + synchronized (this) { + if (threadHashMap.get(threadName) == null) { + HashMap kvHashMap = new HashMap<>(); + V result = kvHashMap.put(key, value); + threadHashMap.put(threadName, kvHashMap); + if (value instanceof Integer) { + if (((Integer) value).intValue() == 0) { + tableStatusList.add(key); + } + } + return result; + } + else { + HashMap kvHashMap = threadHashMap.get(threadName); + V result = kvHashMap.put(key, value); + threadHashMap.put(threadName, kvHashMap); + if (value instanceof Integer) { + if (((Integer) value).intValue() == 0) { + tableStatusList.add(key); + } + } + return result; + } + } + case TABLE_SCHEMA_MAP: + synchronized (this) { + return expiringMap.put(key, value); + } + default: + synchronized (this) { + return hashMap.put(key, value); + } + } + } + + public V put(K key, V value) + { + switch (mapName) { + case TABLE_STATUS_MAP: + synchronized (this) { + String threadName = getThreadName(); + if (threadHashMap.get(threadName) == null) { + HashMap kvHashMap = new HashMap<>(); + V result = kvHashMap.put(key, value); + threadHashMap.put(threadName, kvHashMap); + if (value instanceof Integer) { + if (((Integer) value).intValue() == 0) { + tableStatusList.add(key); + } + } + return result; + } + else { + HashMap kvHashMap = threadHashMap.get(threadName); + V result = kvHashMap.put(key, value); + threadHashMap.put(threadName, kvHashMap); + if (value instanceof Integer) { + if (((Integer) value).intValue() == 0) { + tableStatusList.add(key); + } + } + return result; + } + } + case TABLE_SCHEMA_MAP: + synchronized (this) { + return expiringMap.put(key, value); + } + default: + synchronized (this) { + return hashMap.put(key, value); + } + } + } + + public String getThreadName() + { + String threadName = Thread.currentThread().getName(); + threadName = threadName.substring(0, threadName.lastIndexOf(Const.idSeparator)); + return threadName; + } + + /** + * get put remove + * @param key + * @return + */ + public boolean containsKey(Object key) + { + switch (mapName){ + case TABLE_STATUS_MAP: + synchronized (this) { + String threadName = getThreadName(); + if (!threadHashMap.containsKey(threadName)) { + return false; + } + else { + HashMap kvHashMap = threadHashMap.get(threadName); + boolean result = kvHashMap.containsKey(key); + return result; + } + } + case TABLE_SCHEMA_MAP: + synchronized (this) { + return expiringMap.containsKey(key); + } + default: + synchronized (this) { + return hashMap.containsKey(key); + } + } + } + + public V get(Object key) + { + switch (mapName){ + case TABLE_STATUS_MAP: + synchronized (this) { + String threadName = getThreadName(); + if (!threadHashMap.containsKey(threadName)) { + return null; + } + else { + HashMap kvHashMap = threadHashMap.get(threadName); + V value = kvHashMap.get(key); + return value; + } + } + case TABLE_SCHEMA_MAP: + synchronized (this) { + return expiringMap.get(key); + } + default: + synchronized (this) { + return hashMap.get(key); + } + } + } + + /** + * 仅限于tablestatus开头判断 + * @param key + * @return + */ + public boolean tableStatusKeysExists(Object key) + { + return tableStatusList.contains(key); + } + + public V remove(Object key) + { + switch (mapName){ + case TABLE_STATUS_MAP: + synchronized (this) { + String threadName = getThreadName(); + if (!threadHashMap.containsKey(threadName)) { + return null; + } + else { + HashMap kvHashMap = threadHashMap.get(threadName); + V value = kvHashMap.remove(key); + threadHashMap.remove(threadName); + //可能存在问题 + tableStatusList.remove(key); + return value; + } + } + case TABLE_SCHEMA_MAP: + synchronized (this) { + return expiringMap.remove(key); + } + default: + synchronized (this) { + return hashMap.remove(key); + } + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/TableMoveLock.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/TableMoveLock.java new file mode 100644 index 0000000000000000000000000000000000000000..fec5b74f08e70f44f61a1d4dafc4777786fed816 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/TableMoveLock.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp; + +public class TableMoveLock +{ + private TableMoveLock() + { + } + + public static String getLock(String lock) + { + String internLock = lock.intern(); + return internLock; + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/Scheduler.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/Scheduler.java new file mode 100755 index 0000000000000000000000000000000000000000..9012f20a855a6c8c399e24a517d366e0b7ca7808 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/Scheduler.java @@ -0,0 +1,120 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler; + +import io.airlift.log.Logger; +import io.hetu.core.plugin.mpp.MppConfig; +import io.hetu.core.plugin.mpp.scheduler.db.GsussDBOptThread; +import io.hetu.core.plugin.mpp.scheduler.hadoop.HiveOpt; +import io.hetu.core.plugin.mpp.scheduler.utils.Const; + +import javax.inject.Inject; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; + +public class Scheduler +{ + public static Logger logger = Logger.get(Scheduler.class); + public MppConfig mppConfig; + public Queue> gdsQueue; + private Queue> tmpQueue; + + @Inject + public Scheduler(MppConfig mppConfig) + { + this.mppConfig = mppConfig; + this.gdsQueue = getGdsQueue(mppConfig); + } + + public Queue> getGdsQueue(MppConfig mppConfig) + { + String[] gdsArr = mppConfig.getGdsList().split(","); + Map gdsMaps = new HashMap<>(); + for (String gdsServer : gdsArr) { + gdsServer.split("\\|"); + gdsMaps.put(gdsServer.split("\\|")[0], gdsServer.split("\\|")[1]); + } + tmpQueue = new LinkedList<>(); + for (Map.Entry entry : gdsMaps.entrySet()) { + tmpQueue.add(entry); + } + return tmpQueue; + } + + public Map.Entry getGDS() + { + while (true) { + if (gdsQueue.isEmpty()) { + logger.info("GDS queue is empty, please wait..."); + try { + Thread.sleep(1000); + } + catch (InterruptedException e) { + logger.error(e.getMessage()); + } + } + else { + return gdsQueue.poll(); + } + } + } + + public void prepareHiveExternalTable(Map schemas, String schemaName, String tableName) + { + logger.info("Get schemainfo from gaussDB by table name"); + String tblIdentifier = schemaName + "." + tableName; + + logger.info("Create hive foreign table using alluxio path by hiveserver2 service"); + String auxPath = mppConfig.getAuxUrl() + mppConfig.getBaseAux() + tableName; + HiveOpt.createExternalTable(mppConfig.getHiveUrl() + mppConfig.getHiveDb(), mppConfig.getHiveUser(), mppConfig.getHivePasswd(), + mppConfig.getHsqlDrop(), mppConfig.getHsqlCreate(), + tableName, schemas.get("hiveSchema"), auxPath); + } + + public void startGdsProcess(Map.Entry gdsServer, Map schemas, String schemaName, String tableName) + { + logger.info("Prepare gaussDB GDS process"); + String colSchemaInfo = schemas.get("columns"); + String gsSchemaInfo = schemas.get("gsSchema"); + + String gdsForeignLocation = gdsServer.getKey() + "/" + tableName; + + String createSQL = mppConfig.getGsqlCreate() + .replace("${gaussdb_name}", schemaName) + .replace("${table_name}", tableName) + .replace("${gds_foreign_location}", gdsForeignLocation) + .replace("${table_name}", tableName) + .replace("${schema_info}", gsSchemaInfo); + + String dropSQL = mppConfig.getGsqlDrop() + .replace("${gaussdb_name}", schemaName) + .replace("${table_name}", tableName); + + String insertSQL = mppConfig.getGsqlInsert() + .replace("${gaussdb_name}", schemaName) + .replace("${table_name}", tableName) + .replace("${schema_info}", colSchemaInfo); + String threadName = Const.tableStatus.getThreadName(); + + GsussDBOptThread gsussDBOptThread = new GsussDBOptThread(gdsQueue, gdsServer, + mppConfig.getGsDriver(), mppConfig.getGsUrl(), mppConfig.getGsUser(), mppConfig.getGsPasswd(), + dropSQL, createSQL, insertSQL, schemaName, tableName, mppConfig.getHiveDb(), threadName); + gsussDBOptThread.start(); + logger.info("GaussDB GDS process thread start"); + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOpt.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOpt.java new file mode 100755 index 0000000000000000000000000000000000000000..097ce059965f6fc1081f8037ce289e9f1eebde89 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOpt.java @@ -0,0 +1,128 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.db; + +import io.airlift.log.Logger; +import io.hetu.core.plugin.mpp.MppConfig; +import io.hetu.core.plugin.mpp.scheduler.entity.TableSchema; +import io.hetu.core.plugin.mpp.scheduler.utils.Const; +import io.hetu.core.plugin.mpp.scheduler.utils.Util; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +/** + * @author chengyijian + * @title: GsussDBOpt + * @projectName mpp-scheduler + * @description: GaussDB操作相关 + * @date 2021/8/1210:56 + */ + +public class GsussDBOpt +{ + public static Logger logger = Logger.get(GsussDBOpt.class); + + private GsussDBOpt() + { + } + + public static Connection getConnection(String driver, String url, String username, String passwd) + { + Connection conn = null; + try { + Class.forName(driver).getConstructor().newInstance(); + } + catch (Exception e) { + logger.error(e.getMessage()); + return null; + } + + try { + conn = DriverManager.getConnection(url, username, passwd); + logger.info("GaussDB Connection succeed!"); + } + catch (Exception e) { + logger.error(e.getMessage()); + return null; + } + return conn; + } + + public static Map getSchemas(MppConfig mppConfig, String catalog, String schema, String tableName) + { + String tblIdentifier = catalog + "." + schema + "." + tableName; + + Map schemas = new HashMap<>(); + + if (Const.schemasMap.containsKey(tblIdentifier)) { + TableSchema tableSchema = Const.schemasMap.get(tblIdentifier); + logger.info(tblIdentifier + " schema has stored at " + tableSchema.getSchemaTime()); + schemas.put("columns", tableSchema.getColumns()); + schemas.put("gsSchema", tableSchema.getGsSchema()); + schemas.put("hiveSchema", tableSchema.getHiveSchema()); + return schemas; + } + else { + logger.info(tblIdentifier + " schema has not got it yet!"); + StringBuilder columns; + StringBuilder gsSchema; + StringBuilder hiveSchema; + + Connection conn = GsussDBOpt.getConnection(mppConfig.getGsDriver(), mppConfig.getGsUrl(), mppConfig.getGsUser(), mppConfig.getGsPasswd()); + + try { + DatabaseMetaData dm = conn.getMetaData(); + ResultSet rs = dm.getColumns(catalog, schema, tableName, null); + + columns = new StringBuilder(); + gsSchema = new StringBuilder(); + hiveSchema = new StringBuilder(); + + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + String dataType = rs.getString("TYPE_NAME"); + int columnSize = rs.getInt("COLUMN_SIZE"); + int decimalDigits = rs.getInt("DECIMAL_DIGITS"); + + columns.append(columnName + ","); + gsSchema.append(columnName + " " + Util.getMappingGSType(dataType, columnSize, decimalDigits) + ","); + hiveSchema.append(columnName + " " + Util.getMappingHiveType(dataType, columnSize, decimalDigits) + ","); + } + String columnsTmp = columns.deleteCharAt(columns.length() - 1).toString(); + String gsSchemaTmp = gsSchema.deleteCharAt(gsSchema.length() - 1).toString(); + String hiveSchemaTmp = hiveSchema.deleteCharAt(hiveSchema.length() - 1).toString(); + String schemaTime = Util.getDate(); + + schemas.put("columns", columnsTmp); + schemas.put("gsSchema", gsSchemaTmp); + schemas.put("hiveSchema", hiveSchemaTmp); + + Const.schemasMap.put(tblIdentifier, new TableSchema(columnsTmp, gsSchemaTmp, hiveSchemaTmp, schemaTime)); + rs.close(); + conn.close(); + } + catch (SQLException e) { + logger.error(e.getMessage()); + } + return schemas; + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOptThread.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOptThread.java new file mode 100755 index 0000000000000000000000000000000000000000..3fc87e14f2684959f1dfa08b03e7ad408b3d2937 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/db/GsussDBOptThread.java @@ -0,0 +1,147 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.db; + +import io.airlift.log.Logger; +import io.hetu.core.plugin.mpp.TableMoveLock; +import io.hetu.core.plugin.mpp.scheduler.utils.Const; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.Map; +import java.util.Queue; + +/** + * @author chengyijian + * @title: GsussDBOpt + * @projectName mpp-scheduler + * @description: GaussDB操作相关 + * @date 2021/8/1210:56 + */ +public class GsussDBOptThread + extends Thread +{ + public static Logger logger = Logger.get(GsussDBOptThread.class); + private String driver; + private String jdbcUrl; + private String username; + private String password; + + private String dropSQL; + private String createSQL; + private String insertSQL; + private String gaussdbSchema; + private String tableName; + private String hiveDb; + private String parentThreadName; + + private Queue> gdsQueue; + private Map.Entry gdsServer; + + public GsussDBOptThread(Queue> gdsQueue, Map.Entry gdsServer, + String driver, String jdbcUrl, String username, String password, + String dropSQL, String createSQL, String insertSQL, + String gaussdbSchema, String tableName, String hiveDb, String parentThreadName) + { + super.setName("GsussDBOptThread"); + this.gdsQueue = gdsQueue; + this.gdsServer = gdsServer; + this.driver = driver; + this.jdbcUrl = jdbcUrl; + this.username = username; + this.password = password; + this.dropSQL = dropSQL; + this.createSQL = createSQL; + this.insertSQL = insertSQL; + this.gaussdbSchema = gaussdbSchema; + this.tableName = tableName; + this.hiveDb = hiveDb; + this.parentThreadName = parentThreadName; + } + + public Connection getConnection(String username, String passwd) + { + Connection conn = null; + try { + Class.forName(this.driver).getConstructor().newInstance(); + } + catch (Exception e) { + logger.error(e.getMessage()); + return null; + } + + try { + conn = DriverManager.getConnection(this.jdbcUrl, username, passwd); + logger.info("Connection succeed!"); + } + catch (Exception e) { + logger.error(e.getMessage()); + return null; + } + return conn; + } + + public static void optTable(Connection conn, String sql) + { + Statement stmt = null; + try { + stmt = conn.createStatement(); + stmt.execute(sql); + } + catch (SQLException e) { + logger.error(e.getMessage()); + } + finally { + if (stmt != null) { + try { + stmt.close(); + } + catch (SQLException throwables) { + logger.error(throwables.getMessage()); + } + } + } + } + + @Override + public void run() + { + try { + Connection conn = getConnection(username, password); + logger.info("GaussDB Drop Create Insert Operation Start"); + optTable(conn, dropSQL); + optTable(conn, createSQL); + optTable(conn, insertSQL); + List runningThreadList = Const.runningThreadMap.get(hiveDb + "." + tableName); + synchronized (TableMoveLock.getLock(gaussdbSchema + "." + tableName)) { + for (String threadName : runningThreadList) { + Const.tableStatus.put(hiveDb + "." + tableName, 1, threadName); + Const.runningThreadMap.removeThread(hiveDb + "." + tableName, threadName); + } + } + + logger.info("GaussDB Operation End"); + gdsQueue.add(gdsServer); + logger.info(gdsServer.getKey() + ":" + gdsServer.getValue() + " has been free!"); + conn.close(); + } + catch (SQLException e) { + logger.error(e.getMessage()); + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/ETLInfo.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/ETLInfo.java new file mode 100644 index 0000000000000000000000000000000000000000..d29aa5900fd51ad3b80f09c20c1809d1749c461b --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/ETLInfo.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.entity; + +public class ETLInfo +{ +// status 0:etling 1:etled + private int status; + private String startTime; + private String endTime; + + public ETLInfo() + { + } + + public ETLInfo(int status, String startTime, String endTime) + { + this.status = status; + this.startTime = startTime; + this.endTime = endTime; + } + + public int getStatus() + { + return status; + } + + public void setStatus(int status) + { + this.status = status; + } + + public String getStartTime() + { + return startTime; + } + + public void setStartTime(String startTime) + { + this.startTime = startTime; + } + + public String getEndTime() + { + return endTime; + } + + public void setEndTime(String endTime) + { + this.endTime = endTime; + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/TableSchema.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/TableSchema.java new file mode 100644 index 0000000000000000000000000000000000000000..766d6d30e0f7d5dfa37da1059932208b8c7627d2 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/entity/TableSchema.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.entity; + +public class TableSchema +{ + private String columns; + private String gsSchema; + private String hiveSchema; + private String schemaTime; + + public TableSchema() + { + } + + public TableSchema(String columns, String gsSchema, String hiveSchema, String schemaTime) + { + this.columns = columns; + this.gsSchema = gsSchema; + this.hiveSchema = hiveSchema; + this.schemaTime = schemaTime; + } + + public String getColumns() + { + return columns; + } + + public void setColumns(String columns) + { + this.columns = columns; + } + + public String getGsSchema() + { + return gsSchema; + } + + public void setGsSchema(String gsSchema) + { + this.gsSchema = gsSchema; + } + + public String getHiveSchema() + { + return hiveSchema; + } + + public void setHiveSchema(String hiveSchema) + { + this.hiveSchema = hiveSchema; + } + + public String getSchemaTime() + { + return schemaTime; + } + + public void setSchemaTime(String schemaTime) + { + this.schemaTime = schemaTime; + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/hadoop/HiveOpt.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/hadoop/HiveOpt.java new file mode 100644 index 0000000000000000000000000000000000000000..52eb26758a30b47f21c7ca753ebb332d53ce0ef1 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/hadoop/HiveOpt.java @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.hadoop; + +import io.airlift.log.Logger; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +public class HiveOpt +{ + public static Logger logger = Logger.get(HiveOpt.class); + + private HiveOpt() + { + } + + public static void createExternalTable(String hiveUrl, String hiveUser, String hivePasswd, String hsqlDrop, String hsqlCreate, + String tblName, String schemaInfo, String auxPath) + { + logger.info(tblName + ": Hive External Table has not existed, create it!"); + try { + Class.forName("org.apache.hive.jdbc.HiveDriver"); + Connection connection = DriverManager.getConnection(hiveUrl, hiveUser, hivePasswd); + Statement statement = connection.createStatement(); + String sqlDrop = hsqlDrop.replace("${table_name}", tblName); + String sqlCreate = hsqlCreate + .replace("${table_name}", tblName) + .replace("${schema_info}", schemaInfo) + .replace("${pipe_to_aux_base_path}", auxPath); + statement.execute(sqlDrop); + statement.execute(sqlCreate); + logger.info("Finished create hive foreign table"); + } + catch (ClassNotFoundException e) { + logger.error(e.getMessage()); + } + catch (SQLException e) { + logger.error(e.getMessage()); + } + } +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Const.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Const.java new file mode 100755 index 0000000000000000000000000000000000000000..df4f94df55ddaf04cf2fe5cb833b083b89f9b27d --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Const.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.utils; + +import io.hetu.core.plugin.mpp.RunningTaskHashMap; +import io.hetu.core.plugin.mpp.SynchronizedHashMap; +import io.hetu.core.plugin.mpp.scheduler.entity.ETLInfo; +import io.hetu.core.plugin.mpp.scheduler.entity.TableSchema; + +public class Const +{ + private Const() + { + } + + /** + * Thread Communication + * tableName:String, status:int + * status:0-have created hive table; 1-have finished exported data into table + */ + public static SynchronizedHashMap tableStatus = new SynchronizedHashMap<>("tableStatus"); + + public static String idSeparator = "-"; + + public static SynchronizedHashMap etlInfoMap = new SynchronizedHashMap<>("etlInfo"); + + public static RunningTaskHashMap runningThreadMap = new RunningTaskHashMap(); + + public static SynchronizedHashMap schemasMap = new SynchronizedHashMap<>(100, 10000); +} diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Util.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Util.java new file mode 100755 index 0000000000000000000000000000000000000000..81a24bf540a3a889c2cd47d8f83ee22cc07428f6 --- /dev/null +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/scheduler/utils/Util.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. + * 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 io.hetu.core.plugin.mpp.scheduler.utils; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.regex.Pattern; + +/** + * @author chengyijian + * @title: Util + * @projectName mpp-scheduler + * @description: TODO + * @date 2021/8/1713:54 + */ +public class Util +{ + private Util() + { + } + + public static String getDate() + { + Date date = new Date(); + SimpleDateFormat formatter = new SimpleDateFormat("dd-MM-yyyy HH:mm:ss"); + return formatter.format(date); + } + + /** + * GaussDB 数据类型映射为Hive表的数据类型 + * + * @param gsType + * @return + */ + public static String getMappingHiveType(String gsType, int columnSize, int decimalDigits) + { + String hType = "string"; + + if (Pattern.matches("int.*", gsType)) { + hType = "int"; + } + else if (gsType.equals("float")) { + hType = "float"; + } + else if (gsType.equals("double")) { + hType = "double"; + } + else if (gsType.equals("numeric")) { + hType = "double"; + } + else if (gsType.equals("date")) { + hType = "date"; + } + else if (gsType.equals("bpchar")) { + hType = "string"; + } + else if (gsType.equals("varchar")) { + hType = "string"; + } + return hType; + } + + public static String getMappingGSType(String gsType, int columnSize, int decimalDigits) + { + String hType = "character varying(100)"; + + if (Pattern.matches("int.*", gsType)) { + hType = "integer"; + } + else if (gsType.equals("float")) { + hType = "float"; + } + else if (gsType.equals("double")) { + hType = "double"; + } + else if (gsType.equals("numeric")) { + hType = "numeric(" + columnSize + "," + decimalDigits + ")"; + } + else if (gsType.equals("date")) { + hType = "date"; + } + else if (gsType.equals("bpchar")) { + hType = "character(" + columnSize + ")"; + } + else if (gsType.equals("varchar")) { + hType = "character varying(" + columnSize + ")"; + } + return hType; + } +} diff --git a/hetu-server/src/main/provisio/hetu.xml b/hetu-server/src/main/provisio/hetu.xml index d61f35f6b745cc8d421ced0b817499c26f6b5020..cdc6b07471cfd25f9d4f89b2ac84a8054414f5bc 100644 --- a/hetu-server/src/main/provisio/hetu.xml +++ b/hetu-server/src/main/provisio/hetu.xml @@ -74,6 +74,12 @@ + + + + + + @@ -327,4 +333,9 @@ + + + + + diff --git a/pom.xml b/pom.xml index fa823205cf9d6d1608429e7fb601fab6fcefac08..93be5d0bbc952f9edf8715a60e57ab5070ebc4ba 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,8 @@ hetu-greenplum hetu-clickhouse hetu-kylin + hetu-exchange-filesystem + hetu-mpp @@ -290,6 +292,12 @@ ${project.version} + + io.hetu.core + hetu-mpp + ${project.version} + + io.hetu.core hetu-carbondata @@ -1730,6 +1738,7 @@ ${air.main.basedir}/src/main/resource/license/license-header-alternate-2020.txt ${air.main.basedir}/src/main/resource/license/license-header-alternate-2021.txt ${air.main.basedir}/src/main/resource/license/license-header-alternate-2022.txt + ${air.main.basedir}/src/main/resource/license/license-header-alternate-2022-1.txt ${air.main.basedir}/src/main/resource/license/license-header-third.txt diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitLoader.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitLoader.java index 60314672366baf2e70077642bd5bdb2a21bbff01..c459a30c51bce959af592d939acb9cb181b276ad 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitLoader.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitLoader.java @@ -13,7 +13,7 @@ */ package io.prestosql.plugin.hive; -interface HiveSplitLoader +public interface HiveSplitLoader { void start(HiveSplitSource splitSource); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java index 47fe70ac955b1cd41840a56458d2d9200dcf61d8..0dc7fa85291ffe4442ce5293644fcdfe683dc1fd 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitManager.java @@ -313,7 +313,7 @@ public class HiveSplitManager return highMemorySplitSourceCounter; } - private Iterable getPartitionMetadata(ConnectorSession session, SemiTransactionalHiveMetastore metastore, Table table, SchemaTableName tableName, List hivePartitions, Optional bucketProperty) + public Iterable getPartitionMetadata(ConnectorSession session, SemiTransactionalHiveMetastore metastore, Table table, SchemaTableName tableName, List hivePartitions, Optional bucketProperty) { if (hivePartitions.isEmpty()) { return ImmutableList.of(); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitSource.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitSource.java index 52d8a8f74449d5df26fc46e7ae7c2d8be2b16bba..ed1a09ea5b6f36d7d831a5efdbd597e63a00715f 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitSource.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveSplitSource.java @@ -78,7 +78,7 @@ import static java.lang.Math.toIntExact; import static java.lang.String.format; import static java.util.Objects.requireNonNull; -class HiveSplitSource +public class HiveSplitSource implements ConnectorSplitSource { private static final Logger log = Logger.get(HiveSplitSource.class); diff --git a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveVacuumSplitSource.java b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveVacuumSplitSource.java index c04267b59982b4cbf04c35715e43346c4d7cf296..d3d4fef4cc79c8321915d10827672be5926eade5 100644 --- a/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveVacuumSplitSource.java +++ b/presto-hive/src/main/java/io/prestosql/plugin/hive/HiveVacuumSplitSource.java @@ -47,7 +47,7 @@ import static java.util.concurrent.CompletableFuture.completedFuture; * to same worker and together, to start vacuum operation. So class wrapps all such splits together and scheduled as * single split. */ -class HiveVacuumSplitSource +public class HiveVacuumSplitSource implements ConnectorSplitSource { private HiveSplitSource splitSource; @@ -63,7 +63,7 @@ class HiveVacuumSplitSource private HdfsEnvironment hdfsEnvironment; private HdfsContext hdfsContext; - HiveVacuumSplitSource(HiveSplitSource splitSource, HiveVacuumTableHandle vacuumTableHandle, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, ConnectorSession session) + public HiveVacuumSplitSource(HiveSplitSource splitSource, HiveVacuumTableHandle vacuumTableHandle, HdfsEnvironment hdfsEnvironment, HdfsContext hdfsContext, ConnectorSession session) { this.splitSource = splitSource; this.vacuumTableHandle = vacuumTableHandle; diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorConfig.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorConfig.java index 4a93d5e4079440f834c341e5a1dc84bfc5e9b6e7..41699950bdf58dbf522df625668b58d20a99d0d3 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorConfig.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorConfig.java @@ -70,6 +70,178 @@ public class KafkaConnectorConfig */ private boolean hideInternalColumns = true; + /** + * the path of krb5.conf ,used for develop + */ + private String krb5Conf; + + /** + * the path of kafka_client_jaas_conf + */ + private String loginConfig; + + /** + * whether use subject creds only + */ + private String useSubjectCredsOnly; + + /** + * the group id of kafka + */ + private String groupId; + + /** + * the security protocol of kafka + */ + private String securityProtocol; + + /** + * the sasl mechanism of kafka + */ + private String saslMechanism; + + /** + * the sasl kerberos service name of kafka + */ + private String saslKerberosServiceName; + + /** + * whether to use kerberos + */ + private String kerberosOn; + /** + * whether to use user and password + */ + private String userPasswordOn; + + public String getUserPasswordOn() + { + return userPasswordOn; + } + + @Mandatory(name = "user.password.auth.on", + description = "user.password.auth.on", + defaultValue = "", + required = false) + @Config("user.password.auth.on") + public KafkaConnectorConfig setUserPasswordOn(String userPasswordOn) + { + this.userPasswordOn = userPasswordOn; + return this; + } + + public String getKrb5Conf() + { + return krb5Conf; + } + + @Mandatory(name = "java.security.krb5.conf", + description = "java.security.krb5.conf", + defaultValue = "", + required = false) + @Config("java.security.krb5.conf") + public KafkaConnectorConfig setKrb5Conf(String krb5Conf) + { + this.krb5Conf = krb5Conf; + return this; + } + + public String getLoginConfig() + { + return loginConfig; + } + + @Mandatory(name = "sasl.jaas.config", + description = "sasl.jaas.config", + defaultValue = "", + required = false) + @Config("sasl.jaas.config") + public KafkaConnectorConfig setLoginConfig(String loginConfig) + { + this.loginConfig = loginConfig; + return this; + } + + public String getGroupId() + { + return groupId; + } + + @Mandatory(name = "group.id", + description = "group.id", + defaultValue = "test", + required = false) + @Config("group.id") + public KafkaConnectorConfig setGroupId(String groupId) + { + this.groupId = groupId; + return this; + } + + public String getSecurityProtocol() + { + return securityProtocol; + } + + @Mandatory(name = "security.protocol", + description = "security.protocol", + defaultValue = "SASL_PLAINTEXT", + required = false) + @Config("security.protocol") + public KafkaConnectorConfig setSecurityProtocol(String securityProtocol) + { + this.securityProtocol = securityProtocol; + return this; + } + + public String getSaslMechanism() + { + return saslMechanism; + } + + @Mandatory(name = "sasl.mechanism", + description = "sasl.mechanism", + defaultValue = "GSSAPI", + required = false) + @Config("sasl.mechanism") + public KafkaConnectorConfig setSaslMechanism(String saslMechanism) + { + this.saslMechanism = saslMechanism; + return this; + } + + public String getSaslKerberosServiceName() + { + return saslKerberosServiceName; + } + + @Mandatory(name = "sasl.kerberos.service.name", + description = "sasl.kerberos.service.name", + defaultValue = "kafka", + required = false) + @Config("sasl.kerberos.service.name") + public KafkaConnectorConfig setSaslKerberosServiceName(String saslKerberosServiceName) + { + this.saslKerberosServiceName = saslKerberosServiceName; + return this; + } + + public String isKerberosOn() + { + return kerberosOn; + } + + @Mandatory(name = "kerberos.on", + description = "whether to use kerberos", + defaultValue = "false", + required = false) + @Config("kerberos.on") + public KafkaConnectorConfig setKerberosOn(String kerberosOn) + { + this.kerberosOn = kerberosOn; + return this; + } + @NotNull public File getTableDescriptionDir() { diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java index 5b612ff71c78397f1d0324d233f4f4c430245309..30b1e86d1959783f541d6c02064bd478c933a1fa 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java @@ -27,11 +27,13 @@ import io.prestosql.spi.connector.RecordSet; import io.prestosql.spi.type.Type; import kafka.api.FetchRequest; import kafka.api.FetchRequestBuilder; -import kafka.javaapi.FetchResponse; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.message.MessageAndOffset; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -109,9 +111,9 @@ public class KafkaRecordSet private long totalBytes; private long totalMessages; private long cursorOffset = split.getStart(); - private Iterator messageAndOffsetIterator; + private Iterator> recordIterator; private final AtomicBoolean reported = new AtomicBoolean(); - + private KafkaConsumer leaderKafkaConsumer; private final FieldValueProvider[] currentRowValues = new FieldValueProvider[columnHandles.size()]; KafkaRecordCursor() @@ -147,19 +149,19 @@ public class KafkaRecordSet // Create a fetch request openFetchRequest(); - while (messageAndOffsetIterator.hasNext()) { - MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); - long messageOffset = currentMessageAndOffset.offset(); + while (recordIterator.hasNext()) { + ConsumerRecord record = recordIterator.next(); + long messageOffset = record.offset(); if (messageOffset >= split.getEnd()) { return endOfData(); // Past our split end. Bail. } if (messageOffset >= cursorOffset) { - return nextRow(currentMessageAndOffset); + return nextRow(record); } } - messageAndOffsetIterator = null; + recordIterator = null; } } @@ -173,21 +175,21 @@ public class KafkaRecordSet return false; } - private boolean nextRow(MessageAndOffset messageAndOffset) + private boolean nextRow(ConsumerRecord record) { - cursorOffset = messageAndOffset.offset() + 1; // Cursor now points to the next message. - totalBytes += messageAndOffset.message().payloadSize(); + cursorOffset = record.offset() + 1; // Cursor now points to the next message. + totalBytes += record.serializedValueSize(); totalMessages++; byte[] keyData = EMPTY_BYTE_ARRAY; byte[] messageData = EMPTY_BYTE_ARRAY; - ByteBuffer key = messageAndOffset.message().key(); + ByteBuffer key = record.key(); if (key != null) { keyData = new byte[key.remaining()]; key.get(keyData); } - ByteBuffer message = messageAndOffset.message().payload(); + ByteBuffer message = record.value(); if (message != null) { messageData = new byte[message.remaining()]; message.get(messageData); @@ -206,7 +208,7 @@ public class KafkaRecordSet currentRowValuesMap.put(columnHandle, longValueProvider(totalMessages)); break; case PARTITION_OFFSET_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(messageAndOffset.offset())); + currentRowValuesMap.put(columnHandle, longValueProvider(record.offset())); break; case MESSAGE_FIELD: currentRowValuesMap.put(columnHandle, bytesValueProvider(messageData)); @@ -305,12 +307,15 @@ public class KafkaRecordSet @Override public void close() { + if (leaderKafkaConsumer != null) { + leaderKafkaConsumer.close(); + } } private void openFetchRequest() { try { - if (messageAndOffsetIterator == null) { + if (recordIterator == null) { log.debug("Fetching %d bytes from offset %d (%d - %d). %d messages read so far", KAFKA_READ_BUFFER_SIZE, cursorOffset, split.getStart(), split.getEnd(), totalMessages); FetchRequest req = new FetchRequestBuilder() .clientId("presto-worker-" + Thread.currentThread().getName()) @@ -319,16 +324,14 @@ public class KafkaRecordSet // TODO - this should look at the actual node this is running on and prefer // that copy if running locally. - look into NodeInfo - SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); - - FetchResponse fetchResponse = consumer.fetch(req); - if (fetchResponse.hasError()) { - short errorCode = fetchResponse.errorCode(split.getTopicName(), split.getPartitionId()); - log.warn("Fetch response has error: %d", errorCode); - throw new RuntimeException("could not fetch data from Kafka, error code is '" + errorCode + "'"); + if (leaderKafkaConsumer == null) { + leaderKafkaConsumer = consumerManager.getSaslConsumer(split.getLeader()); } - - messageAndOffsetIterator = fetchResponse.messageSet(split.getTopicName(), split.getPartitionId()).iterator(); + TopicPartition topicPartition = new TopicPartition(split.getTopicName(), split.getPartitionId()); + leaderKafkaConsumer.assign(Collections.singletonList(topicPartition)); + leaderKafkaConsumer.seek(topicPartition, cursorOffset); + ConsumerRecords records = leaderKafkaConsumer.poll(500); + recordIterator = records.records(topicPartition).iterator(); } } catch (Exception e) { // Catch all exceptions because Kafka library is written in scala and checked exceptions are not declared in method signature. diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSimpleConsumerManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSimpleConsumerManager.java index 1f082360be5f65a3439c2f2762ee8c021f29ae68..1584b0fc7b918c0b4f41302ef950c812f122174f 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSimpleConsumerManager.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSimpleConsumerManager.java @@ -20,11 +20,15 @@ import io.airlift.log.Logger; import io.prestosql.spi.HostAddress; import io.prestosql.spi.NodeManager; import kafka.javaapi.consumer.SimpleConsumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; import javax.annotation.PreDestroy; import javax.inject.Inject; +import java.nio.ByteBuffer; import java.util.Map; +import java.util.Properties; +import java.util.UUID; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; @@ -43,6 +47,15 @@ public class KafkaSimpleConsumerManager private final int connectTimeoutMillis; private final int bufferSizeBytes; + private final String kerberosOn; + private final String loginConfig; + private final String krb5Conf; + private String groupId; + private final String securityProtocol; + private final String saslMechanism; + private final String saslKerberosServiceName; + private final String userPasswordOn; + @Inject public KafkaSimpleConsumerManager( KafkaConnectorConfig kafkaConnectorConfig, @@ -55,6 +68,15 @@ public class KafkaSimpleConsumerManager this.bufferSizeBytes = toIntExact(kafkaConnectorConfig.getKafkaBufferSize().toBytes()); this.consumerCache = CacheBuilder.newBuilder().build(CacheLoader.from(this::createConsumer)); + + this.kerberosOn = kafkaConnectorConfig.isKerberosOn(); + this.userPasswordOn = kafkaConnectorConfig.getUserPasswordOn(); + this.loginConfig = kafkaConnectorConfig.getLoginConfig(); + this.krb5Conf = kafkaConnectorConfig.getKrb5Conf(); + this.groupId = kafkaConnectorConfig.getGroupId(); + this.securityProtocol = kafkaConnectorConfig.getSecurityProtocol(); + this.saslMechanism = kafkaConnectorConfig.getSaslMechanism(); + this.saslKerberosServiceName = kafkaConnectorConfig.getSaslKerberosServiceName(); } @PreDestroy @@ -76,6 +98,12 @@ public class KafkaSimpleConsumerManager return consumerCache.getUnchecked(host); } + public KafkaConsumer getSaslConsumer(HostAddress host) + { + requireNonNull(host, "host is null"); + return createSaslConsumer(host); + } + private SimpleConsumer createConsumer(HostAddress host) { log.info("Creating new Consumer for %s", host); @@ -85,4 +113,40 @@ public class KafkaSimpleConsumerManager bufferSizeBytes, "presto-kafka-" + nodeManager.getCurrentNode().getNodeIdentifier()); } + + private KafkaConsumer createSaslConsumer(HostAddress host) + { + log.info("Creating new SaslConsumer for %s", host); + Properties props = new Properties(); + if ("true".equalsIgnoreCase(kerberosOn)) { + props.put("sasl.jaas.config", loginConfig); + System.setProperty("java.security.krb5.conf", krb5Conf); + props.put("security.protocol", securityProtocol); + props.put("sasl.mechanism", saslMechanism); + props.put("sasl.kerberos.service.name", saslKerberosServiceName); + } + else if ("true".equalsIgnoreCase(userPasswordOn)) { + props.put("sasl.jaas.config", loginConfig); + props.put("security.protocol", securityProtocol); + props.put("sasl.mechanism", saslMechanism); + } + + try { + props.put("bootstrap.servers", host.toString()); + props.put("enable.auto.commit", "false"); + props.put("key.deserializer", Class.forName("org.apache.kafka.common.serialization.ByteBufferDeserializer")); + props.put("value.deserializer", Class.forName("org.apache.kafka.common.serialization.ByteBufferDeserializer")); + if (groupId == null) { + groupId = UUID.randomUUID().toString(); + } + props.put("group.id", groupId); + props.put("session.timeout.ms", connectTimeoutMillis); + props.put("receive.buffer.bytes", bufferSizeBytes); + } + catch (ClassNotFoundException e) { + log.error(e, "failed to create kafka consumer"); + } + + return new KafkaConsumer<>(props); + } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java index be74c05c371fb25c5dd71c5db559c6597af1845a..eb480cbcc1bb44ea7f8c91581a0bca90229e5b30 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java @@ -28,15 +28,14 @@ import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.connector.FixedSplitSource; import kafka.api.PartitionOffsetRequestInfo; -import kafka.cluster.BrokerEndPoint; import kafka.common.TopicAndPartition; import kafka.javaapi.OffsetRequest; import kafka.javaapi.OffsetResponse; -import kafka.javaapi.PartitionMetadata; -import kafka.javaapi.TopicMetadata; -import kafka.javaapi.TopicMetadataRequest; -import kafka.javaapi.TopicMetadataResponse; import kafka.javaapi.consumer.SimpleConsumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import javax.inject.Inject; @@ -47,6 +46,7 @@ import java.io.InputStreamReader; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; +import java.nio.ByteBuffer; import java.util.List; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; @@ -84,44 +84,30 @@ public class KafkaSplitManager public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, SplitSchedulingStrategy splitSchedulingStrategy) { KafkaTableHandle kafkaTableHandle = (KafkaTableHandle) table; - try { - SimpleConsumer simpleConsumer = consumerManager.getConsumer(selectRandom(nodes)); - - TopicMetadataRequest topicMetadataRequest = new TopicMetadataRequest(ImmutableList.of(kafkaTableHandle.getTopicName())); - TopicMetadataResponse topicMetadataResponse = simpleConsumer.send(topicMetadataRequest); + try (KafkaConsumer kafkaConsumer = consumerManager.getSaslConsumer(selectRandom(nodes))) { + List partitionInfos = kafkaConsumer.partitionsFor(kafkaTableHandle.getTopicName()); ImmutableList.Builder splits = ImmutableList.builder(); - for (TopicMetadata metadata : topicMetadataResponse.topicsMetadata()) { - for (PartitionMetadata part : metadata.partitionsMetadata()) { - log.debug("Adding Partition %s/%s", metadata.topic(), part.partitionId()); - - BrokerEndPoint leader = part.leader(); - if (leader == null) { - throw new PrestoException(GENERIC_INTERNAL_ERROR, format("Leader election in progress for Kafka topic '%s' partition %s", metadata.topic(), part.partitionId())); - } - - HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); - - SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); - // Kafka contains a reverse list of "end - start" pairs for the splits - - long[] offsets = findAllOffsets(leaderConsumer, metadata.topic(), part.partitionId()); - - for (int i = offsets.length - 1; i > 0; i--) { - KafkaSplit split = new KafkaSplit( - metadata.topic(), - kafkaTableHandle.getKeyDataFormat(), - kafkaTableHandle.getMessageDataFormat(), - kafkaTableHandle.getKeyDataSchemaLocation().map(KafkaSplitManager::readSchema), - kafkaTableHandle.getMessageDataSchemaLocation().map(KafkaSplitManager::readSchema), - part.partitionId(), - offsets[i], - offsets[i - 1], - partitionLeader); - splits.add(split); - } - } + for (PartitionInfo partitionInfo : partitionInfos) { + log.debug("Adding Partition %s/%s", partitionInfo.topic(), partitionInfo.partition()); + Node leader = partitionInfo.leader(); + HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); + TopicPartition topicPartition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition()); + kafkaConsumer.assign(ImmutableList.of(topicPartition)); + long beginOffset = kafkaConsumer.beginningOffsets(ImmutableList.of(topicPartition)).values().iterator().next(); + long endOffset = kafkaConsumer.endOffsets(ImmutableList.of(topicPartition)).values().iterator().next(); + KafkaSplit split = new KafkaSplit( + topicPartition.topic(), + kafkaTableHandle.getKeyDataFormat(), + kafkaTableHandle.getMessageDataFormat(), + kafkaTableHandle.getKeyDataSchemaLocation().map(KafkaSplitManager::readSchema), + kafkaTableHandle.getMessageDataSchemaLocation().map(KafkaSplitManager::readSchema), + topicPartition.partition(), + beginOffset, + endOffset, + partitionLeader); + splits.add(split); } return new FixedSplitSource(splits.build()); diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConnectorConfig.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConnectorConfig.java index 27650796b65c7388fb6ee296bd9c1bc5d95a0475..9adc647e77b9225e3f961858e380ed63d676305a 100644 --- a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConnectorConfig.java +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConnectorConfig.java @@ -32,6 +32,14 @@ public class TestKafkaConnectorConfig .setDefaultSchema("default") .setTableNames("") .setTableDescriptionDir(new File("etc/kafka/")) + .setGroupId(null) + .setKerberosOn(null) + .setSecurityProtocol(null) + .setKrb5Conf(null) + .setUserPasswordOn(null) + .setLoginConfig(null) + .setSaslKerberosServiceName(null) + .setSaslMechanism(null) .setHideInternalColumns(true)); } @@ -46,6 +54,19 @@ public class TestKafkaConnectorConfig .put("kafka.connect-timeout", "1h") .put("kafka.buffer-size", "1MB") .put("kafka.hide-internal-columns", "false") + .put("group.id", "test") + .put("sasl.jaas.config", "com.sun.security.auth.module.Krb5LoginModule required" + + " useKeyTab=true" + + " useTicketCache=true" + + " serviceName=kafka" + + " keyTab=\"/Users/mac/Desktop/user01.keytab\"" + + " principal=\"user01@EXAMPLE.COM\";") + .put("java.security.krb5.conf", "/etc/krb5.conf") + .put("kerberos.on", "false") + .put("user.password.auth.on", "false") + .put("sasl.kerberos.service.name", "kafka") + .put("sasl.mechanism", "GSSAPI") + .put("security.protocol", "SASL_PLAINTEXT") .build(); KafkaConnectorConfig expected = new KafkaConnectorConfig() @@ -55,6 +76,19 @@ public class TestKafkaConnectorConfig .setNodes("localhost:12345, localhost:23456") .setKafkaConnectTimeout("1h") .setKafkaBufferSize("1MB") + .setGroupId("test") + .setKrb5Conf("/etc/krb5.conf") + .setLoginConfig("com.sun.security.auth.module.Krb5LoginModule required" + + " useKeyTab=true" + + " useTicketCache=true" + + " serviceName=kafka" + + " keyTab=\"/Users/mac/Desktop/user01.keytab\"" + + " principal=\"user01@EXAMPLE.COM\";") + .setSaslKerberosServiceName("kafka") + .setSaslMechanism("GSSAPI") + .setKerberosOn("false") + .setUserPasswordOn("false") + .setSecurityProtocol("SASL_PLAINTEXT") .setHideInternalColumns(false); ConfigAssertions.assertFullMapping(properties, expected); diff --git a/presto-main/etc/catalog/kafka.properties b/presto-main/etc/catalog/kafka.properties new file mode 100644 index 0000000000000000000000000000000000000000..92acf61327e112bfb3c87202449f9b89ffe14a32 --- /dev/null +++ b/presto-main/etc/catalog/kafka.properties @@ -0,0 +1,12 @@ +connector.name=kafka +kafka.nodes=localhost:9092 +kafka.table-names=testTopic +kafka.hide-internal-columns=false +kerberos.on=false +java.security.krb5.conf=/Users/mac/Desktop/krb5.conf +sasl.jaas.config= com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true useTicketCache=true serviceName=kafka keyTab=\"/Users/mac/Desktop/user01.keytab\" principal=\"user01@EXAMPLE.COM\"; +group.id=testTopic +security.protocol=SASL_PLAINTEXT +sasl.mechanism=GSSAPI +sasl.kerberos.service.name=kafka +user.password.auth.on = false \ No newline at end of file diff --git a/presto-main/etc/catalog/mpp.properties b/presto-main/etc/catalog/mpp.properties new file mode 100644 index 0000000000000000000000000000000000000000..d7d6387fa4ec0fb1ba94302501d8138f9c6eb1f6 --- /dev/null +++ b/presto-main/etc/catalog/mpp.properties @@ -0,0 +1,30 @@ + +connector.name=mpp +hive.metastore.uri=thrift://localhost:9083 + +#GDS baseinfo +gds-list=gsfs://localhost:port1|base_path +aux-url=alluxio://localhost:19998 +base-aux=/gdsdata/ +etl-reuse=false + +#hive info +hive-user=username +hive-passwd=password +hive-db=xxx_db + +hive-url=jdbc:hive2://localhost:10000/ +# hive template +hsql-drop=drop table if exists ${table_name} +hsql-create=CREATE EXTERNAL TABLE ${table_name} ( ${schema_info} ) COMMENT 'gds external table' ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\n' LOCATION '${pipe_to_aux_base_path}' + +# gsDB connection info +gs-driver=org.postgresql.Driver +gs-url=jdbc:postgresql://localhost:25308/schema +gs-user=user +gs-passwd=password + +# gaussdb template +gsql-create=create foreign table ${gaussdb_name}.ext_${table_name} ( ${schema_info} ) SERVER gsmpp_server OPTIONS ( LOCATION '${gds_foreign_location}', FORMAT 'text', DELIMITER E',', NULL '', encoding 'UTF-8', noescaping 'true', EOL E'\\n', out_filename_prefix '${table_name}') WRITE ONLY; +gsql-insert=insert into ${gaussdb_name}.ext_${table_name} select ${schema_info} from ${gaussdb_name}.${table_name}; +gsql-drop=drop foreign table if exists ${gaussdb_name}.ext_${table_name}; diff --git a/presto-main/etc/config.properties b/presto-main/etc/config.properties index a821a2a2387eb882d33f890d06343f960273dfb9..4439c46a09e4947032749ba645f2b195b6696524 100644 --- a/presto-main/etc/config.properties +++ b/presto-main/etc/config.properties @@ -55,6 +55,7 @@ plugin.bundles=\ ../presto-memory/pom.xml,\ ../presto-jmx/pom.xml,\ ../presto-hive-hadoop2/pom.xml,\ + ../hetu-mpp/pom.xml,\ ../presto-example-http/pom.xml,\ ../presto-kafka/pom.xml, \ ../presto-tpch/pom.xml, \ diff --git a/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerModule.java b/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerModule.java new file mode 100644 index 0000000000000000000000000000000000000000..295c77604e8e60b648ce3e8662e4676ca2fd4afc --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerModule.java @@ -0,0 +1,25 @@ +/* + * 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 io.prestosql.exchange; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +public class ExchangeManagerModule implements Module { + @Override + public void configure(Binder binder) { + binder.bind(ExchangeManagerRegistry.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerRegistry.java b/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerRegistry.java new file mode 100644 index 0000000000000000000000000000000000000000..755a7ed5d922a06f6c4774d34cf2038edd487171 --- /dev/null +++ b/presto-main/src/main/java/io/prestosql/exchange/ExchangeManagerRegistry.java @@ -0,0 +1,96 @@ +/* + * 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 io.prestosql.exchange; + +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.prestosql.metadata.ExchangeHandleResolver; +import io.prestosql.spi.classloader.ThreadContextClassLoader; +import io.prestosql.spi.exchange.ExchangeManager; +import io.prestosql.spi.exchange.ExchangeManagerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.isNullOrEmpty; +import static io.airlift.configuration.ConfigurationLoader.loadPropertiesFrom; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class ExchangeManagerRegistry { + private static final Logger log = Logger.get(ExchangeManagerRegistry.class); + + private static final File CONFIG_FILE = new File("etc/exchange-manager.properties"); + private static final String EXCHANGE_MANAGER_NAME_PROPERTY = "exchange-manager.name"; + + private final ExchangeHandleResolver handleResolver; + private final Map exchangeManagerFactories = new ConcurrentHashMap<>(); + private volatile ExchangeManager exchangeManager; + + @Inject + public ExchangeManagerRegistry(ExchangeHandleResolver handleResolver) { + this.handleResolver = requireNonNull(handleResolver, "handleResolver is null"); + } + + public void addExchangeManagerFactory(ExchangeManagerFactory exchangeManagerFactory) { + requireNonNull(exchangeManagerFactory, "exchangeManagerFactory is null"); + if (exchangeManagerFactories.putIfAbsent(exchangeManagerFactory.getName(), exchangeManagerFactory) != null) { + throw new IllegalArgumentException(format("Exchange manager factory '%s' is already registered", exchangeManagerFactory.getName())); + } + } + + public void loadExchangeManager() { + if (!CONFIG_FILE.exists()) { + return; + } + try { + Map properties = loadPropertiesFrom(CONFIG_FILE.getPath()); + String name = properties.remove(EXCHANGE_MANAGER_NAME_PROPERTY); + checkArgument(!isNullOrEmpty(name), "Exchange manager configuration %s does not contain %s", CONFIG_FILE, EXCHANGE_MANAGER_NAME_PROPERTY); + } catch (IOException e) { + throw new UncheckedIOException("Failed to read configuration file: " + CONFIG_FILE, e); + } + } + + public synchronized void loadExchangeManager(String name, Map properties) { + log.info("-- Loading exchange manager %s --", name); + + checkState(exchangeManager == null, "exchangeManager is already loaded"); + ExchangeManagerFactory factory = exchangeManagerFactories.get(name); + checkArgument(factory != null, "Exchange manager factory '%s' is not registered. Available factories: %s", name, exchangeManagerFactories.keySet()); + + ExchangeManager exchangeManager; + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(factory.getClass().getClassLoader())) { + exchangeManager = factory.create(properties); + } + handleResolver.setExchangeManagerHandleResolver(factory.getHandleResolver()); + + log.info("-- Loaded exchange manager %s --", name); + this.exchangeManager = exchangeManager; + } + + private Map loadProperties(File configFile) { + try { + return new HashMap<>(loadPropertiesFrom(configFile.getPath())); + } catch (IOException e) { + throw new UncheckedIOException("Failed to read configuration file: " + configFile, e); + } + } +} diff --git a/presto-main/src/main/java/io/prestosql/metadata/HandleJsonModule.java b/presto-main/src/main/java/io/prestosql/metadata/HandleJsonModule.java index 18b1797b54536c9f0f57b64cad907708f97d5f1f..28de720691319023d54a65762fb955516d1d26db 100644 --- a/presto-main/src/main/java/io/prestosql/metadata/HandleJsonModule.java +++ b/presto-main/src/main/java/io/prestosql/metadata/HandleJsonModule.java @@ -29,6 +29,7 @@ import io.prestosql.spi.connector.ConnectorTableLayoutHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.connector.ConnectorUpdateTableHandle; import io.prestosql.spi.connector.ConnectorVacuumTableHandle; +import io.prestosql.spi.exchange.ExchangeHandleResolver; public class HandleJsonModule implements Module @@ -37,6 +38,7 @@ public class HandleJsonModule public void configure(Binder binder) { binder.bind(HandleResolver.class).in(Scopes.SINGLETON); + binder.bind(ExchangeHandleResolver.class).in(Scopes.SINGLETON); } @ProvidesIntoSet diff --git a/presto-main/src/main/java/io/prestosql/server/PluginManager.java b/presto-main/src/main/java/io/prestosql/server/PluginManager.java index 3f9ff137ee7535a4d118b72f67d7651a35ea56c1..545a8e6f13b96fd6156c24f85f998112fa66cf53 100644 --- a/presto-main/src/main/java/io/prestosql/server/PluginManager.java +++ b/presto-main/src/main/java/io/prestosql/server/PluginManager.java @@ -22,6 +22,7 @@ import io.airlift.resolver.DefaultArtifact; import io.prestosql.connector.ConnectorManager; import io.prestosql.cube.CubeManager; import io.prestosql.eventlistener.EventListenerManager; +import io.prestosql.exchange.ExchangeManagerRegistry; import io.prestosql.execution.resourcegroups.ResourceGroupManager; import io.prestosql.failuredetector.FailureDetectorManager; import io.prestosql.failuredetector.FailureDetectorPlugin; @@ -40,6 +41,7 @@ import io.prestosql.spi.classloader.ThreadContextClassLoader; import io.prestosql.spi.connector.ConnectorFactory; import io.prestosql.spi.cube.CubeProvider; import io.prestosql.spi.eventlistener.EventListenerFactory; +import io.prestosql.spi.exchange.ExchangeManagerFactory; import io.prestosql.spi.failuredetector.FailureRetryFactory; import io.prestosql.spi.filesystem.HetuFileSystemClientFactory; import io.prestosql.spi.function.FunctionNamespaceManagerFactory; @@ -115,6 +117,9 @@ public class PluginManager private final FileSystemClientManager fileSystemClientManager; private final FailureDetectorManager failureDetectorManager; private final HeuristicIndexerManager heuristicIndexerManager; + + private final ExchangeManagerRegistry exchangeManagerRegistry; + private final SessionPropertyDefaults sessionPropertyDefaults; private final ArtifactResolver resolver; private final File installedPluginsDir; @@ -142,7 +147,8 @@ public class PluginManager FileSystemClientManager fileSystemClientManager, HetuMetaStoreManager hetuMetaStoreManager, HeuristicIndexerManager heuristicIndexerManager, - FailureDetectorManager failureDetectorManager) + FailureDetectorManager failureDetectorManager, + ExchangeManagerRegistry exchangeManagerRegistry) { requireNonNull(nodeInfo, "nodeInfo is null"); requireNonNull(config, "config is null"); @@ -175,6 +181,26 @@ public class PluginManager this.hetuMetaStoreManager = requireNonNull(hetuMetaStoreManager, "hetuMetaStoreManager is null"); this.heuristicIndexerManager = requireNonNull(heuristicIndexerManager, "heuristicIndexerManager is null"); this.failureDetectorManager = requireNonNull(failureDetectorManager, "failureDetectorManager is null"); + this.exchangeManagerRegistry = requireNonNull(exchangeManagerRegistry, "exchangeManagerRegistry is null"); + } + + private static List listFiles(File installedPluginsDir) + { + if (installedPluginsDir != null && installedPluginsDir.isDirectory()) { + File[] files = installedPluginsDir.listFiles(); + if (files != null) { + Arrays.sort(files); + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private static List sortedArtifacts(List artifacts) + { + List list = new ArrayList<>(artifacts); + Collections.sort(list, Ordering.natural().nullsLast().onResultOf(Artifact::getFile)); + return list; } public void loadPlugins() @@ -357,6 +383,11 @@ public class PluginManager FailureDetectorManager.addFailureRetryFactory(failureRetryFactory); } + for (ExchangeManagerFactory exchangeManagerFactory : plugin.getExchangeManagerFactories()) { + log.info("Registering exchange manager %s", exchangeManagerFactory.getName()); + exchangeManagerRegistry.addExchangeManagerFactory(exchangeManagerFactory); + } + installFunctionsPlugin(plugin); } @@ -429,23 +460,4 @@ public class PluginManager ClassLoader parent = getClass().getClassLoader(); return new PluginClassLoader(urls, parent, SPI_PACKAGES); } - - private static List listFiles(File installedPluginsDir) - { - if (installedPluginsDir != null && installedPluginsDir.isDirectory()) { - File[] files = installedPluginsDir.listFiles(); - if (files != null) { - Arrays.sort(files); - return ImmutableList.copyOf(files); - } - } - return ImmutableList.of(); - } - - private static List sortedArtifacts(List artifacts) - { - List list = new ArrayList<>(artifacts); - Collections.sort(list, Ordering.natural().nullsLast().onResultOf(Artifact::getFile)); - return list; - } } diff --git a/presto-main/src/main/java/io/prestosql/testing/LocalQueryRunner.java b/presto-main/src/main/java/io/prestosql/testing/LocalQueryRunner.java index 47450f16bcd7df3066e7358dcdba10c38550ee3d..6eb67453b9cd6e3ceac263a0f1f832b99ed2f83d 100644 --- a/presto-main/src/main/java/io/prestosql/testing/LocalQueryRunner.java +++ b/presto-main/src/main/java/io/prestosql/testing/LocalQueryRunner.java @@ -50,6 +50,7 @@ import io.prestosql.cost.TaskCountEstimator; import io.prestosql.cube.CubeManager; import io.prestosql.dynamicfilter.DynamicFilterCacheManager; import io.prestosql.eventlistener.EventListenerManager; +import io.prestosql.exchange.ExchangeManagerRegistry; import io.prestosql.execution.CommentTask; import io.prestosql.execution.CommitTask; import io.prestosql.execution.CreateTableTask; @@ -128,6 +129,7 @@ import io.prestosql.spi.Plugin; import io.prestosql.spi.connector.CatalogName; import io.prestosql.spi.connector.ConnectorFactory; import io.prestosql.spi.connector.QualifiedObjectName; +import io.prestosql.spi.exchange.ExchangeHandleResolver; import io.prestosql.spi.failuredetector.FailureRetryPolicy; import io.prestosql.spi.metadata.TableHandle; import io.prestosql.spi.operator.ReuseExchangeOperator; @@ -455,7 +457,8 @@ public class LocalQueryRunner fileSystemClientManager, hetuMetaStoreManager, heuristicIndexerManager, - failureDetectorManager); + failureDetectorManager, + new ExchangeManagerRegistry(new ExchangeHandleResolver())); connectorManager.addConnectorFactory(globalSystemConnectorFactory); connectorManager.createConnection(GlobalSystemConnector.NAME, GlobalSystemConnector.NAME, ImmutableMap.of()); diff --git a/presto-main/src/main/resources/webapp/dist/overview.js b/presto-main/src/main/resources/webapp/dist/overview.js index ebc3c0eeb162b2069dc5566ad1a0fbfc09fd5106..92d5c592e11a86c8af70f28542b401c6bb7f9207 100644 --- a/presto-main/src/main/resources/webapp/dist/overview.js +++ b/presto-main/src/main/resources/webapp/dist/overview.js @@ -26310,7 +26310,7 @@ eval("\n\nvar _react = __webpack_require__(/*! react */ \"./node_modules/react/i /***/ (function(module, exports, __webpack_require__) { "use strict"; -eval("\n\nObject.defineProperty(exports, \"__esModule\", {\n value: true\n});\n\nvar _createClass = function () { function defineProperties(target, props) { for (var i = 0; i < props.length; i++) { var descriptor = props[i]; descriptor.enumerable = descriptor.enumerable || false; descriptor.configurable = true; if (\"value\" in descriptor) descriptor.writable = true; Object.defineProperty(target, descriptor.key, descriptor); } } return function (Constructor, protoProps, staticProps) { if (protoProps) defineProperties(Constructor.prototype, protoProps); if (staticProps) defineProperties(Constructor, staticProps); return Constructor; }; }();\n\nvar _react = __webpack_require__(/*! react */ \"./node_modules/react/index.js\");\n\nvar _react2 = _interopRequireDefault(_react);\n\nvar _echarts = __webpack_require__(/*! echarts/lib/echarts */ \"./node_modules/echarts/lib/echarts.js\");\n\nvar _echarts2 = _interopRequireDefault(_echarts);\n\n__webpack_require__(/*! echarts/lib/chart/line */ \"./node_modules/echarts/lib/chart/line.js\");\n\n__webpack_require__(/*! echarts/lib/chart/treemap */ \"./node_modules/echarts/lib/chart/treemap.js\");\n\n__webpack_require__(/*! echarts/theme/royal */ \"./node_modules/echarts/theme/royal.js\");\n\n__webpack_require__(/*! echarts/lib/component/tooltip */ \"./node_modules/echarts/lib/component/tooltip.js\");\n\n__webpack_require__(/*! echarts/lib/component/title */ \"./node_modules/echarts/lib/component/title.js\");\n\nvar _OverviewActions = __webpack_require__(/*! ./OverviewActions */ \"./overview/OverviewActions.js\");\n\nvar _OverviewActions2 = _interopRequireDefault(_OverviewActions);\n\nvar _OverviewStore = __webpack_require__(/*! ./OverviewStore */ \"./overview/OverviewStore.js\");\n\nvar _OverviewStore2 = _interopRequireDefault(_OverviewStore);\n\nvar _reactSimpleMultiSelect = __webpack_require__(/*! react-simple-multi-select */ \"./node_modules/react-simple-multi-select/build/components/MultiSelect.js\");\n\nvar _reactSimpleMultiSelect2 = _interopRequireDefault(_reactSimpleMultiSelect);\n\nvar _utils = __webpack_require__(/*! ../utils */ \"./utils.js\");\n\nvar _lodash = __webpack_require__(/*! lodash */ \"./node_modules/lodash/lodash.js\");\n\nvar _lodash2 = _interopRequireDefault(_lodash);\n\nfunction _interopRequireDefault(obj) { return obj && obj.__esModule ? obj : { default: obj }; }\n\nfunction _toConsumableArray(arr) { if (Array.isArray(arr)) { for (var i = 0, arr2 = Array(arr.length); i < arr.length; i++) { arr2[i] = arr[i]; } return arr2; } else { return Array.from(arr); } }\n\nfunction _classCallCheck(instance, Constructor) { if (!(instance instanceof Constructor)) { throw new TypeError(\"Cannot call a class as a function\"); } }\n\nfunction _possibleConstructorReturn(self, call) { if (!self) { throw new ReferenceError(\"this hasn't been initialised - super() hasn't been called\"); } return call && (typeof call === \"object\" || typeof call === \"function\") ? call : self; }\n\nfunction _inherits(subClass, superClass) { if (typeof superClass !== \"function\" && superClass !== null) { throw new TypeError(\"Super expression must either be null or a function, not \" + typeof superClass); } subClass.prototype = Object.create(superClass && superClass.prototype, { constructor: { value: subClass, enumerable: false, writable: true, configurable: true } }); if (superClass) Object.setPrototypeOf ? Object.setPrototypeOf(subClass, superClass) : subClass.__proto__ = superClass; } /*\n * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved.\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nvar EchartPart = function (_React$Component) {\n _inherits(EchartPart, _React$Component);\n\n function EchartPart(props) {\n _classCallCheck(this, EchartPart);\n\n var _this = _possibleConstructorReturn(this, (EchartPart.__proto__ || Object.getPrototypeOf(EchartPart)).call(this, props));\n\n _this.state = {\n checkStatus: {\n checkOne: true,\n checkTwo: true,\n checkThree: true,\n checkFour: true,\n checkFive: true,\n checkSix: true,\n checkSeven: true,\n heatMapChart: true,\n cpuLoad: true,\n heatMapMemoryChart: true\n },\n itemList: [{ key: \"Cluster CPU Usage\", value: \"heatMapChart\" }, { key: \"Cluster Free Memory\", value: \"heatMapMemoryChart\" }, { key: \"Avg Cluster CPU Usage\", value: \"cpuLoad\" }, { key: \"Used Query Memory\", value: \"checkOne\" }, { key: \"Running Queries\", value: \"checkTwo\" }, { key: \"Queued Queries\", value: \"checkThree\" }, { key: \"Blocked Queries\", value: \"checkFour\" }, { key: \"Active Workers\", value: \"checkFive\" }, { key: \"Avg Running Tasks\", value: \"checkSix\" }, { key: \"Avg CPU cycles per worker\", value: \"checkSeven\" }],\n selectedItemList: [{ key: \"Cluster CPU Usage\", value: \"heatMapChart\" }, { key: \"Cluster Free Memory\", value: \"heatMapMemoryChart\" }, { key: \"Avg Cluster CPU Usage\", value: \"cpuLoad\" }, { key: \"Used Query Memory\", value: \"checkOne\" }, { key: \"Running Queries\", value: \"checkTwo\" }, { key: \"Queued Queries\", value: \"checkThree\" }, { key: \"Blocked Queries\", value: \"checkFour\" }, { key: \"Active Workers\", value: \"checkFive\" }, { key: \"Avg Running Tasks\", value: \"checkSix\" }, { key: \"Avg CPU cycles per worker\", value: \"checkSeven\" }],\n chartName: ['Used Query Memory', 'Running Queries', 'Queued Queries', 'Blocked Queries', 'Active Workers', 'Avg Running Tasks', 'Avg CPU cycles per worker'],\n step: 10,\n timer: null,\n chartCpu: [],\n heatMapChart: [],\n heatMapMemoryChart: [],\n chart1: [],\n chart2: [],\n chart3: [],\n chart4: [],\n chart5: [],\n chart6: [],\n chart7: [],\n chartRef: null,\n lastRow: null,\n lastByte: null,\n lastWorker: null,\n memoryInit: false,\n unitArr: ['bytes', 'quantity', 'quantity', 'quantity', 'quantity', 'quantity', 'quantity'],\n lastRefresh: null\n };\n _this.state.chartRef = Object.keys(_this.state.checkStatus), _this._onChange = _this._onChange.bind(_this);\n _this.changeList = _this.changeList.bind(_this);\n _this.resize = _this.resize.bind(_this);\n return _this;\n }\n\n _createClass(EchartPart, [{\n key: \"resize\",\n value: function resize() {\n for (var i = 0; i < this.state.chartRef.length; i++) {\n var ref = this.refs[this.state.chartRef[i]];\n if (!ref.className) {\n var chart = _echarts2.default.init(ref);\n chart.resize({ silent: true });\n }\n }\n }\n }, {\n key: \"changeList\",\n value: function changeList(selectedItemList) {\n var _this2 = this;\n\n this.state.itemList.map(function (item) {\n _this2.state.checkStatus[item.value] = false;\n });\n selectedItemList.map(function (item) {\n _this2.state.checkStatus[item.value] = true;\n });\n var state = this.state;\n state.selectedItemList = selectedItemList;\n this.setState(state);\n }\n }, {\n key: \"changeState\",\n value: function changeState(name) {\n var state = this.state;\n state.checkStatus[name] = !state.checkStatus[name];\n this.setState(state);\n }\n\n //echarts\n\n }, {\n key: \"componentDidMount\",\n value: function componentDidMount() {\n this.setXAxis();\n _OverviewActions2.default.getData();\n _OverviewActions2.default.getMemoryData();\n _OverviewStore2.default.listen(this._onChange);\n this.lineDatas();\n\n var win = window;\n if (win.addEventListener) {\n win.addEventListener('resize', this.resize, false);\n } else if (win.attachEvent) {\n win.attachEvent('onresize', this.resize);\n } else {\n win.onresize = this.resize;\n }\n $(window).on('resize', this.resize);\n }\n }, {\n key: \"componentWillUnmount\",\n value: function componentWillUnmount() {\n _OverviewStore2.default.unlisten(this._onChange);\n clearInterval(this.state.timer);\n }\n\n //obtained data per sec\n\n }, {\n key: \"lineDatas\",\n value: function lineDatas() {\n this.state.timer = setInterval(function () {\n _OverviewActions2.default.getData();\n _OverviewActions2.default.getMemoryData();\n }, 1000);\n }\n //refresh line\n\n }, {\n key: \"_onChange\",\n value: function _onChange(data) {\n if (data.requestNum % 2 === 0) {\n if (!this.state.memoryInit && data.memoryData) {\n // let cpuChart=echarts.init(this.refs.cpuLoad);\n // let option=cpuChart.getOption();\n // let memoryInitData=[];\n // let cpuSeries={};\n // let index = 0;\n // Object.keys(data.memoryData).map(key=>{\n // let op = Object.assign({}, option.series[index]);\n // index++;\n // op.name = key.slice(0, key.indexOf(\" \"));\n // let currentCpuData = [...this.delete(this.state.chartCpu), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]];\n // op.data = this.state.step === 10 ? currentCpuData.slice(1200) : this.state.step === 20 ? currentCpuData.slice(600) : currentCpuData;\n // op.areaStyle = {\n // shadowBlur: 10,\n // opacity: 0.1\n // };\n // op.type = 'line';\n // op.showSymbol = false;\n // memoryInitData.push(op);\n // cpuSeries[key]= currentCpuData;\n // })\n // option.series=memoryInitData;\n // option.yAxis = {max: 100, min: 0, type: \"value\"};\n // cpuChart.setOption(option);\n\n var _heatMapChart = _echarts2.default.init(this.refs.heatMapChart, \"royal\");\n _heatMapChart.setOption({\n animation: false,\n title: {\n text: 'Cluster CPU Usage',\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'item',\n formatter: function formatter(params, t, cb) {\n return params.name + \" : \" + params.value + \"%\";\n }\n },\n series: [{\n type: 'treemap',\n data: this.state.heatMapChart\n }]\n });\n var _heatMapMemoryChart = _echarts2.default.init(this.refs.heatMapMemoryChart, \"royal\");\n _heatMapMemoryChart.setOption({\n animation: false,\n title: {\n text: 'Cluster Free Memory ',\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'item',\n formatter: function formatter(params, t, cb) {\n return params.name + \" : \" + (0, _utils.formatDataSizeBytes)(params.value);\n }\n },\n series: [{\n type: 'treemap',\n data: this.state.heatMapMemoryChart\n }]\n });\n\n this.setState({\n memoryInit: true\n });\n }\n // else{\n // let dataCpu=this.state.chartCpu;\n // let mychart1=echarts.init(this.refs.cpuLoad);\n // let option=mychart1.getOption();\n // let memoryInitData=option.series;\n // Object.keys(data.memoryData).map(key=>{\n // let dataCpuElement = dataCpu[key];\n // if (_.isUndefined(dataCpuElement)) {\n // let op = Object.assign({}, option.series[index]);\n // op.name = key.slice(0, key.indexOf(\" \"));\n // op.areaStyle = {\n // shadowBlur: 10,\n // opacity: 0.1\n // };\n // op.type = 'line';\n // dataCpu[key] = [...this.delete(dataCpuElement), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]];\n // op.data = dataCpu[key];\n // memoryInitData.push(op);\n // }\n // else {\n // dataCpu[key] = [...this.delete(dataCpuElement), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]];\n // }\n // for(let i=0,len=memoryInitData.length;i= 600) {\n dataset = dataset.splice(600 - 1, dataset.length - 600 - 1);\n }\n dataset = [].concat(_toConsumableArray(dataset), [newDataPoint]);\n entry.dataset = dataset;\n var sum = 0;\n for (var i = 0; i < dataset.length; i++) {\n sum += dataset[i];\n }\n entry.value = Number((sum / dataset.length).toFixed(2));\n }\n });\n\n var heatMapDataSort = (0, _utils.bubbleSort)(heatMapData);\n this.state.heatMapChart = heatMapDataSort.slice(0, 10);\n var heatMapChart = _echarts2.default.init(this.refs.heatMapChart, \"royal\");\n var heatMapChartOption = heatMapChart.getOption();\n heatMapChartOption.series = [{\n type: \"treemap\",\n data: heatMapDataSort.slice(0, 10),\n breadcrumb: {\n show: false\n }\n }];\n heatMapChart.setOption(heatMapChartOption);\n\n //heatMap memory data\n var heatMapMemoryData = this.state.heatMapMemoryChart;\n Object.keys(data.memoryData).map(function (key) {\n var id = data.memoryData[key].id;\n var name = key;\n var index = _lodash2.default.findIndex(heatMapMemoryData, { id: id });\n var newDataPoint = 0;\n if (typeof data.memoryData[key].pools.general != \"undefined\") {\n newDataPoint += data.memoryData[key].pools.general.freeBytes;\n if (typeof data.memoryData[key].pools.reserved != \"undefined\") {\n newDataPoint += data.memoryData[key].pools.reserved.freeBytes;\n }\n }\n newDataPoint = Number(newDataPoint);\n if (index == -1) {\n var newData = {};\n newData.id = id;\n newData.name = name;\n newData.value = newDataPoint;\n newData.dataset = [newDataPoint];\n newData.children = [];\n heatMapMemoryData.push(newData);\n } else {\n var entry = heatMapMemoryData[index];\n var dataset = entry.dataset;\n if (dataset.length >= 600) {\n dataset = dataset.splice(600 - 1, dataset.length - 600 - 1);\n }\n dataset = [].concat(_toConsumableArray(dataset), [newDataPoint]);\n entry.dataset = dataset;\n var sum = 0;\n for (var i = 0; i < dataset.length; i++) {\n sum += dataset[i];\n }\n entry.value = Number((sum / dataset.length).toFixed(2));\n }\n });\n var heatMapMemoryDataSort = (0, _utils.bubbleSort)(heatMapMemoryData);\n this.state.heatMapMemoryChart = heatMapMemoryDataSort.slice(0, 10);\n var heatMapMemoryChart = _echarts2.default.init(this.refs.heatMapMemoryChart, \"royal\");\n var heatMapMemoryChartOption = heatMapMemoryChart.getOption();\n heatMapMemoryChartOption.series = [{\n type: \"treemap\",\n data: heatMapMemoryDataSort.slice(0, 10),\n breadcrumb: {\n show: false\n }\n }];\n heatMapMemoryChart.setOption(heatMapMemoryChartOption);\n\n var now = Date.now();\n var secondsSinceLastRefresh = this.state.lastRefresh ? (now - this.state.lastRefresh) / 1000.0 : 1;\n secondsSinceLastRefresh = secondsSinceLastRefresh < 1 ? 1 : secondsSinceLastRefresh;\n var lastWorker = this.state.lastWorker ? (data.lineData.totalCpuTimeSecs - this.state.lastWorker) / data.lineData.activeWorkers / secondsSinceLastRefresh : 0;\n this.setState({\n chartCpu: [].concat(_toConsumableArray(this.delete(this.state.chartCpu)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), (data.lineData.systemCpuLoad * 100).toFixed(4)]]),\n chart1: [].concat(_toConsumableArray(this.delete(this.state.chart1)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.reservedMemory]]),\n chart2: [].concat(_toConsumableArray(this.delete(this.state.chart2)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.runningQueries]]),\n chart3: [].concat(_toConsumableArray(this.delete(this.state.chart3)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.queuedQueries]]),\n chart4: [].concat(_toConsumableArray(this.delete(this.state.chart4)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.blockedQueries]]),\n chart5: [].concat(_toConsumableArray(this.delete(this.state.chart5)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.activeWorkers]]),\n chart6: [].concat(_toConsumableArray(this.delete(this.state.chart6)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.runningDrivers]]),\n chart7: [].concat(_toConsumableArray(this.delete(this.state.chart7)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), lastWorker]]),\n lastWorker: data.lineData.totalCpuTimeSecs,\n heatMapChart: this.state.heatMapChart,\n heatMapMemoryChart: this.state.heatMapMemoryChart,\n lastRefresh: now\n });\n if (!this.refs.cpuLoad.className) {\n var mychart = _echarts2.default.init(this.refs.cpuLoad);\n var option = mychart.getOption();\n option.series[0].data = this.state.step === 10 ? this.state.chartCpu.slice(1200) : this.state.step === 20 ? this.state.chartCpu.slice(600) : this.state.chartCpu;\n option.series[0].areaStyle = {\n color: \"#41BB04\",\n shadowBlur: 10,\n opacity: 0.1\n };\n option.series[0].lineStyle = { color: \"#137113\" };\n option.series[0].itemStyle = { color: \"#137113\" };\n option.yAxis = { max: 100, min: 0, type: \"value\" };\n mychart.setOption(option);\n }\n for (var i = 0; i < this.state.chartName.length; i++) {\n if (!this.refs[this.state.chartRef[i]].className) {\n var _mychart = _echarts2.default.init(this.refs[this.state.chartRef[i]]);\n var _option = _mychart.getOption();\n _option.series[0].data = this.state.step === 10 ? this.state['chart' + parseInt(i + 1)].slice(1200) : this.state.step === 20 ? this.state['chart' + parseInt(i + 1)].slice(600) : this.state['chart' + parseInt(i + 1)];\n _option.series[0].areaStyle = {\n color: \"#c3c683\",\n shadowBlur: 10,\n opacity: 0.1\n };\n _option.series[0].lineStyle = { color: \"#b6a019\" };\n _option.series[0].itemStyle = { color: \"#b6a019\" };\n _mychart.setOption(_option);\n }\n }\n }\n }\n\n // delete first data\n\n }, {\n key: \"delete\",\n value: function _delete(arr) {\n if (_lodash2.default.isUndefined(arr)) {\n return [];\n }\n arr.splice(0, 1);\n return arr;\n }\n //according to step to set XAxis data\n\n }, {\n key: \"setXAxis\",\n value: function setXAxis() {\n var arr = [];\n for (var i = 0, len = 30 * 60; i < len; i++) {\n arr[i] = [new Date(new Date().getTime() - 1000 * i).format('yyyy-MM-dd hh:mm:ss'), 0];\n }\n arr = arr.reverse();\n this.setState({\n chartCpu: [].concat(_toConsumableArray(arr)),\n chart1: [].concat(_toConsumableArray(arr)),\n chart2: [].concat(_toConsumableArray(arr)),\n chart3: [].concat(_toConsumableArray(arr)),\n chart4: [].concat(_toConsumableArray(arr)),\n chart5: [].concat(_toConsumableArray(arr)),\n chart6: [].concat(_toConsumableArray(arr)),\n chart7: [].concat(_toConsumableArray(arr))\n });\n var mychart1 = _echarts2.default.init(this.refs.cpuLoad);\n mychart1.setOption({\n animation: false,\n title: { text: 'Average Cluster CPU Usage',\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'axis'\n },\n xAxis: {\n type: 'time',\n name: 'time',\n interval: 60 * 1000 * this.state.step / 10,\n boundaryGap: false,\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n var date = new Date(value).format(\"yyyy-MM-dd hh:mm:ss\");\n return date.slice(11, 16);\n }\n }\n },\n yAxis: {\n name: 'usage(%)',\n axisTick: {\n show: false\n },\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n return value;\n }\n }\n },\n series: [{\n type: 'line',\n symbol: 'none',\n data: []\n }]\n });\n for (var _i = 0; _i < this.state.chartName.length; _i++) {\n if (!this.refs[this.state.chartRef[_i]].className) {\n var mychart = _echarts2.default.init(this.refs[this.state.chartRef[_i]]);\n mychart.setOption({\n animation: false,\n title: {\n text: this.state.chartName[_i],\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'axis'\n },\n xAxis: {\n type: 'time',\n name: 'time',\n interval: 60 * 1000 * this.state.step / 10,\n boundaryGap: false,\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n var date = new Date(value).format(\"yyyy-MM-dd hh:mm:ss\");\n return date.slice(11, 16);\n }\n }\n },\n yAxis: {\n name: this.state.unitArr[_i],\n axisTick: {\n show: false\n },\n axisLabel: {\n formatter: function (name, value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n if (name === 'quantity') {\n return (0, _utils.formatCount)(value);\n } else if (name === 'bytes') {\n return (0, _utils.formatDataSizeBytes)(value);\n } else {\n return value;\n }\n }.bind(null, this.state.unitArr[_i])\n }\n },\n series: [{\n type: 'line',\n symbol: 'none',\n data: this.state.step === 10 ? this.state['chart' + parseInt(_i + 1)].slice(1200) : this.state.step === 20 ? this.state['chart' + parseInt(_i + 1)].slice(600) : this.state['chart' + parseInt(_i + 1)]\n }]\n });\n }\n }\n }\n }, {\n key: \"selected\",\n value: function selected(e) {\n clearInterval(this.state.timer);\n e.preventDefault();\n var val = e.target.selectedIndex === 0 ? 10 : e.target.selectedIndex === 1 ? 20 : 30;\n var state = this.state;\n state.step = val;\n this.setState(state);\n for (var i = 0; i < this.state.chartName.length; i++) {\n if (!this.refs[this.state.chartRef[i]].className) {\n var mychart = _echarts2.default.init(this.refs[this.state.chartRef[i]]);\n var _option2 = mychart.getOption();\n _option2.xAxis[0].interval = 60 * 1000 * this.state.step / 10;\n // option.series[0].data=[];\n mychart.setOption(_option2);\n }\n }\n var mychart1 = _echarts2.default.init(this.refs.cpuLoad);\n var option = mychart1.getOption();\n option.xAxis[0].interval = 60 * 1000 * this.state.step / 10;\n mychart1.setOption(option);\n _OverviewActions2.default.getData();\n this.lineDatas();\n }\n }, {\n key: \"render\",\n value: function render() {\n var _this3 = this;\n\n var style = { height: \"30vh\", width: \"calc(40vw - 80px)\", left: \"center\", top: \"center\" };\n return _react2.default.createElement(\n \"div\",\n null,\n _react2.default.createElement(\n \"div\",\n { className: \"selectItemContainer\" },\n _react2.default.createElement(\n \"div\",\n { className: \"selectChart multiSelect\" },\n _react2.default.createElement(_reactSimpleMultiSelect2.default, {\n title: \"Select Chart\",\n itemList: this.state.itemList,\n selectedItemList: this.state.selectedItemList,\n changeList: this.changeList,\n isObjectArray: true\n })\n ),\n _react2.default.createElement(\n \"div\",\n { className: \"select-part\" },\n _react2.default.createElement(\n \"select\",\n { onChange: this.selected.bind(this), value: this.state.step },\n _react2.default.createElement(\n \"option\",\n { value: \"10\" },\n \"Last 10 minutes\"\n ),\n _react2.default.createElement(\n \"option\",\n { value: \"20\" },\n \"Last 20 minutes\"\n ),\n _react2.default.createElement(\n \"option\",\n { value: \"30\" },\n \"Last 30 minutes\"\n )\n )\n )\n ),\n _react2.default.createElement(\n \"div\",\n { className: \"overviewGraphContainerParent\" },\n _react2.default.createElement(\n \"div\",\n { className: \"overviewGraphContainer\" },\n _react2.default.createElement(\n \"div\",\n { className: this.state.checkStatus[\"heatMapChart\"] ? 'overviewChart' : 'display-none' },\n _react2.default.createElement(\"div\", { ref: \"heatMapChart\", style: style })\n ),\n _react2.default.createElement(\n \"div\",\n { className: this.state.checkStatus[\"heatMapMemoryChart\"] ? 'overviewChart' : 'display-none' },\n _react2.default.createElement(\"div\", { ref: \"heatMapMemoryChart\", style: style })\n ),\n _react2.default.createElement(\n \"div\",\n { className: this.state.checkStatus[\"cpuLoad\"] ? 'overviewChart' : 'display-none' },\n _react2.default.createElement(\"div\", { ref: \"cpuLoad\", style: style })\n ),\n Object.keys(this.state.checkStatus).map(function (key, index) {\n if (key == 'cpuLoad' || key == 'heatMapChart' || key == 'heatMapMemoryChart') {\n return null;\n }\n return _react2.default.createElement(\n \"div\",\n { className: _this3.state.checkStatus[key] ? 'overviewChart' : 'display-none', key: index },\n _react2.default.createElement(\"div\", { ref: key, style: style })\n );\n })\n )\n )\n );\n }\n }]);\n\n return EchartPart;\n}(_react2.default.Component);\n\nexports.default = EchartPart;\n\n//# sourceURL=webpack:///./overview/EchartPart.jsx?"); +eval("\n\nObject.defineProperty(exports, \"__esModule\", {\n value: true\n});\n\nvar _createClass = function () { function defineProperties(target, props) { for (var i = 0; i < props.length; i++) { var descriptor = props[i]; descriptor.enumerable = descriptor.enumerable || false; descriptor.configurable = true; if (\"value\" in descriptor) descriptor.writable = true; Object.defineProperty(target, descriptor.key, descriptor); } } return function (Constructor, protoProps, staticProps) { if (protoProps) defineProperties(Constructor.prototype, protoProps); if (staticProps) defineProperties(Constructor, staticProps); return Constructor; }; }();\n\nvar _react = __webpack_require__(/*! react */ \"./node_modules/react/index.js\");\n\nvar _react2 = _interopRequireDefault(_react);\n\nvar _echarts = __webpack_require__(/*! echarts/lib/echarts */ \"./node_modules/echarts/lib/echarts.js\");\n\nvar _echarts2 = _interopRequireDefault(_echarts);\n\n__webpack_require__(/*! echarts/lib/chart/line */ \"./node_modules/echarts/lib/chart/line.js\");\n\n__webpack_require__(/*! echarts/lib/chart/treemap */ \"./node_modules/echarts/lib/chart/treemap.js\");\n\n__webpack_require__(/*! echarts/theme/royal */ \"./node_modules/echarts/theme/royal.js\");\n\n__webpack_require__(/*! echarts/lib/component/tooltip */ \"./node_modules/echarts/lib/component/tooltip.js\");\n\n__webpack_require__(/*! echarts/lib/component/title */ \"./node_modules/echarts/lib/component/title.js\");\n\nvar _OverviewActions = __webpack_require__(/*! ./OverviewActions */ \"./overview/OverviewActions.js\");\n\nvar _OverviewActions2 = _interopRequireDefault(_OverviewActions);\n\nvar _OverviewStore = __webpack_require__(/*! ./OverviewStore */ \"./overview/OverviewStore.js\");\n\nvar _OverviewStore2 = _interopRequireDefault(_OverviewStore);\n\nvar _reactSimpleMultiSelect = __webpack_require__(/*! react-simple-multi-select */ \"./node_modules/react-simple-multi-select/build/components/MultiSelect.js\");\n\nvar _reactSimpleMultiSelect2 = _interopRequireDefault(_reactSimpleMultiSelect);\n\nvar _utils = __webpack_require__(/*! ../utils */ \"./utils.js\");\n\nvar _lodash = __webpack_require__(/*! lodash */ \"./node_modules/lodash/lodash.js\");\n\nvar _lodash2 = _interopRequireDefault(_lodash);\n\nfunction _interopRequireDefault(obj) { return obj && obj.__esModule ? obj : { default: obj }; }\n\nfunction _toConsumableArray(arr) { if (Array.isArray(arr)) { for (var i = 0, arr2 = Array(arr.length); i < arr.length; i++) { arr2[i] = arr[i]; } return arr2; } else { return Array.from(arr); } }\n\nfunction _classCallCheck(instance, Constructor) { if (!(instance instanceof Constructor)) { throw new TypeError(\"Cannot call a class as a function\"); } }\n\nfunction _possibleConstructorReturn(self, call) { if (!self) { throw new ReferenceError(\"this hasn't been initialised - super() hasn't been called\"); } return call && (typeof call === \"object\" || typeof call === \"function\") ? call : self; }\n\nfunction _inherits(subClass, superClass) { if (typeof superClass !== \"function\" && superClass !== null) { throw new TypeError(\"Super expression must either be null or a function, not \" + typeof superClass); } subClass.prototype = Object.create(superClass && superClass.prototype, { constructor: { value: subClass, enumerable: false, writable: true, configurable: true } }); if (superClass) Object.setPrototypeOf ? Object.setPrototypeOf(subClass, superClass) : subClass.__proto__ = superClass; } /*\n * Copyright (C) 2018-2021. Huawei Technologies Co., Ltd. All rights reserved.\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nvar EchartPart = function (_React$Component) {\n _inherits(EchartPart, _React$Component);\n\n function EchartPart(props) {\n _classCallCheck(this, EchartPart);\n\n var _this = _possibleConstructorReturn(this, (EchartPart.__proto__ || Object.getPrototypeOf(EchartPart)).call(this, props));\n\n _this.state = {\n checkStatus: {\n checkOne: true,\n checkTwo: true,\n checkThree: true,\n checkFour: true,\n checkFive: true,\n checkSix: true,\n checkSeven: true,\n checkEight: true,\n cpuLoad: true\n },\n itemList: [{ key: \"Avg Cluster CPU Usage\", value: \"cpuLoad\" }, { key: \"Used Query Memory\", value: \"checkOne\" }, { key: \"Running Queries\", value: \"checkTwo\" }, { key: \"Queued Queries\", value: \"checkThree\" }, { key: \"Blocked Queries\", value: \"checkFour\" }, { key: \"Active Workers\", value: \"checkFive\" }, { key: \"Avg Running Tasks\", value: \"checkSix\" }, { key: \"Avg CPU cycles per worker\", value: \"checkSeven\" }, { key: \"Input Total Bytes\", value: \"checkEight\" }],\n selectedItemList: [{ key: \"Avg Cluster CPU Usage\", value: \"cpuLoad\" }, { key: \"Used Query Memory\", value: \"checkOne\" }, { key: \"Running Queries\", value: \"checkTwo\" }, { key: \"Queued Queries\", value: \"checkThree\" }, { key: \"Blocked Queries\", value: \"checkFour\" }, { key: \"Active Workers\", value: \"checkFive\" }, { key: \"Avg Running Tasks\", value: \"checkSix\" }, { key: \"Avg CPU cycles per worker\", value: \"checkSeven\" }, { key: \"total Input Bytes\", value: \"checkEight\" }],\n chartName: ['Used Query Memory', 'Running Queries', 'Queued Queries', 'Blocked Queries', 'Active Workers', 'Avg Running Tasks', 'Avg CPU cycles per worker', 'total Input Bytes'],\n step: 10,\n timer: null,\n chartCpu: [],\n chart1: [],\n chart2: [],\n chart3: [],\n chart4: [],\n chart5: [],\n chart6: [],\n chart7: [],\n chart8: [],\n chartRef: null,\n lastRow: null,\n lastByte: null,\n lastWorker: null,\n memoryInit: false,\n unitArr: ['bytes', 'quantity', 'quantity', 'quantity', 'quantity', 'quantity', 'quantity', 'bytes'],\n lastRefresh: null\n };\n _this.state.chartRef = Object.keys(_this.state.checkStatus), _this._onChange = _this._onChange.bind(_this);\n _this.changeList = _this.changeList.bind(_this);\n _this.resize = _this.resize.bind(_this);\n return _this;\n }\n\n _createClass(EchartPart, [{\n key: \"resize\",\n value: function resize() {\n for (var i = 0; i < this.state.chartRef.length; i++) {\n var ref = this.refs[this.state.chartRef[i]];\n if (!ref.className) {\n var chart = _echarts2.default.init(ref);\n chart.resize({ silent: true });\n }\n }\n }\n }, {\n key: \"changeList\",\n value: function changeList(selectedItemList) {\n var _this2 = this;\n\n this.state.itemList.map(function (item) {\n _this2.state.checkStatus[item.value] = false;\n });\n selectedItemList.map(function (item) {\n _this2.state.checkStatus[item.value] = true;\n });\n var state = this.state;\n state.selectedItemList = selectedItemList;\n this.setState(state);\n }\n }, {\n key: \"changeState\",\n value: function changeState(name) {\n var state = this.state;\n state.checkStatus[name] = !state.checkStatus[name];\n this.setState(state);\n }\n\n //echarts\n\n }, {\n key: \"componentDidMount\",\n value: function componentDidMount() {\n this.setXAxis();\n _OverviewActions2.default.getData();\n _OverviewActions2.default.getMemoryData();\n _OverviewStore2.default.listen(this._onChange);\n this.lineDatas();\n\n var win = window;\n if (win.addEventListener) {\n win.addEventListener('resize', this.resize, false);\n } else if (win.attachEvent) {\n win.attachEvent('onresize', this.resize);\n } else {\n win.onresize = this.resize;\n }\n $(window).on('resize', this.resize);\n }\n }, {\n key: \"componentWillUnmount\",\n value: function componentWillUnmount() {\n _OverviewStore2.default.unlisten(this._onChange);\n clearInterval(this.state.timer);\n }\n\n //obtained data per sec\n\n }, {\n key: \"lineDatas\",\n value: function lineDatas() {\n this.state.timer = setInterval(function () {\n _OverviewActions2.default.getData();\n _OverviewActions2.default.getMemoryData();\n }, 1000);\n }\n //refresh line\n\n }, {\n key: \"_onChange\",\n value: function _onChange(data) {\n if (data.requestNum % 2 === 0) {\n if (!this.state.memoryInit && data.memoryData) {\n this.setState({\n memoryInit: true\n });\n }\n var now = Date.now();\n var secondsSinceLastRefresh = this.state.lastRefresh ? (now - this.state.lastRefresh) / 1000.0 : 1;\n secondsSinceLastRefresh = secondsSinceLastRefresh < 1 ? 1 : secondsSinceLastRefresh;\n var lastWorker = this.state.lastWorker ? (data.lineData.totalCpuTimeSecs - this.state.lastWorker) / data.lineData.activeWorkers / secondsSinceLastRefresh : 0;\n this.setState({\n chartCpu: [].concat(_toConsumableArray(this.delete(this.state.chartCpu)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), (data.lineData.systemCpuLoad * 100).toFixed(4)]]),\n chart1: [].concat(_toConsumableArray(this.delete(this.state.chart1)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.reservedMemory]]),\n chart2: [].concat(_toConsumableArray(this.delete(this.state.chart2)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.runningQueries]]),\n chart3: [].concat(_toConsumableArray(this.delete(this.state.chart3)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.queuedQueries]]),\n chart4: [].concat(_toConsumableArray(this.delete(this.state.chart4)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.blockedQueries]]),\n chart5: [].concat(_toConsumableArray(this.delete(this.state.chart5)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.activeWorkers]]),\n chart6: [].concat(_toConsumableArray(this.delete(this.state.chart6)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.runningDrivers]]),\n chart7: [].concat(_toConsumableArray(this.delete(this.state.chart7)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), lastWorker]]),\n chart8: [].concat(_toConsumableArray(this.delete(this.state.chart8)), [[new Date().format('yyyy-MM-dd hh:mm:ss'), data.lineData.totalInputBytes]]),\n lastWorker: data.lineData.totalCpuTimeSecs,\n lastRefresh: now\n });\n if (!this.refs.cpuLoad.className) {\n var mychart = _echarts2.default.init(this.refs.cpuLoad);\n var option = mychart.getOption();\n option.series[0].data = this.state.step === 10 ? this.state.chartCpu.slice(1200) : this.state.step === 20 ? this.state.chartCpu.slice(600) : this.state.chartCpu;\n option.series[0].areaStyle = {\n color: \"#41BB04\",\n shadowBlur: 10,\n opacity: 0.1\n };\n option.series[0].lineStyle = { color: \"#137113\" };\n option.series[0].itemStyle = { color: \"#137113\" };\n option.yAxis = { max: 100, min: 0, type: \"value\" };\n mychart.setOption(option);\n }\n for (var i = 0; i < this.state.chartName.length; i++) {\n if (!this.refs[this.state.chartRef[i]].className) {\n var _mychart = _echarts2.default.init(this.refs[this.state.chartRef[i]]);\n var _option = _mychart.getOption();\n _option.series[0].data = this.state.step === 10 ? this.state['chart' + parseInt(i + 1)].slice(1200) : this.state.step === 20 ? this.state['chart' + parseInt(i + 1)].slice(600) : this.state['chart' + parseInt(i + 1)];\n _option.series[0].areaStyle = {\n color: \"#c3c683\",\n shadowBlur: 10,\n opacity: 0.1\n };\n _option.series[0].lineStyle = { color: \"#b6a019\" };\n _option.series[0].itemStyle = { color: \"#b6a019\" };\n _mychart.setOption(_option);\n }\n }\n }\n }\n\n // delete first data\n\n }, {\n key: \"delete\",\n value: function _delete(arr) {\n if (_lodash2.default.isUndefined(arr)) {\n return [];\n }\n arr.splice(0, 1);\n return arr;\n }\n //according to step to set XAxis data\n\n }, {\n key: \"setXAxis\",\n value: function setXAxis() {\n var arr = [];\n for (var i = 0, len = 30 * 60; i < len; i++) {\n arr[i] = [new Date(new Date().getTime() - 1000 * i).format('yyyy-MM-dd hh:mm:ss'), 0];\n }\n arr = arr.reverse();\n this.setState({\n chartCpu: [].concat(_toConsumableArray(arr)),\n chart1: [].concat(_toConsumableArray(arr)),\n chart2: [].concat(_toConsumableArray(arr)),\n chart3: [].concat(_toConsumableArray(arr)),\n chart4: [].concat(_toConsumableArray(arr)),\n chart5: [].concat(_toConsumableArray(arr)),\n chart6: [].concat(_toConsumableArray(arr)),\n chart7: [].concat(_toConsumableArray(arr)),\n chart8: [].concat(_toConsumableArray(arr))\n });\n var mychart1 = _echarts2.default.init(this.refs.cpuLoad);\n mychart1.setOption({\n animation: false,\n title: { text: 'Average Cluster CPU Usage',\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'axis'\n },\n xAxis: {\n type: 'time',\n name: 'time',\n interval: 60 * 1000 * this.state.step / 10,\n boundaryGap: false,\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n var date = new Date(value).format(\"yyyy-MM-dd hh:mm:ss\");\n return date.slice(11, 16);\n }\n }\n },\n yAxis: {\n name: 'usage(%)',\n axisTick: {\n show: false\n },\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n return value;\n }\n }\n },\n series: [{\n type: 'line',\n symbol: 'none',\n data: []\n }]\n });\n for (var _i = 0; _i < this.state.chartName.length; _i++) {\n if (!this.refs[this.state.chartRef[_i]].className) {\n var mychart = _echarts2.default.init(this.refs[this.state.chartRef[_i]]);\n mychart.setOption({\n animation: false,\n title: {\n text: this.state.chartName[_i],\n left: 'center',\n textStyle: {\n color: \"#767676\",\n fontSize: 16\n }\n },\n tooltip: {\n trigger: 'axis'\n },\n xAxis: {\n type: 'time',\n name: 'time',\n interval: 60 * 1000 * this.state.step / 10,\n boundaryGap: false,\n axisLabel: {\n formatter: function formatter(value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n var date = new Date(value).format(\"yyyy-MM-dd hh:mm:ss\");\n return date.slice(11, 16);\n }\n }\n },\n yAxis: {\n name: this.state.unitArr[_i],\n axisTick: {\n show: false\n },\n axisLabel: {\n formatter: function (name, value, index) {\n if (index % 2 == 1) {\n return \"\";\n }\n if (name === 'quantity') {\n return (0, _utils.formatCount)(value);\n } else if (name === 'bytes') {\n return (0, _utils.formatDataSizeBytes)(value);\n } else {\n return value;\n }\n }.bind(null, this.state.unitArr[_i])\n }\n },\n series: [{\n type: 'line',\n symbol: 'none',\n data: this.state.step === 10 ? this.state['chart' + parseInt(_i + 1)].slice(1200) : this.state.step === 20 ? this.state['chart' + parseInt(_i + 1)].slice(600) : this.state['chart' + parseInt(_i + 1)]\n }]\n });\n }\n }\n }\n }, {\n key: \"selected\",\n value: function selected(e) {\n clearInterval(this.state.timer);\n e.preventDefault();\n var val = e.target.selectedIndex === 0 ? 10 : e.target.selectedIndex === 1 ? 20 : 30;\n var state = this.state;\n state.step = val;\n this.setState(state);\n for (var i = 0; i < this.state.chartName.length; i++) {\n if (!this.refs[this.state.chartRef[i]].className) {\n var mychart = _echarts2.default.init(this.refs[this.state.chartRef[i]]);\n var _option2 = mychart.getOption();\n _option2.xAxis[0].interval = 60 * 1000 * this.state.step / 10;\n mychart.setOption(_option2);\n }\n }\n var mychart1 = _echarts2.default.init(this.refs.cpuLoad);\n var option = mychart1.getOption();\n option.xAxis[0].interval = 60 * 1000 * this.state.step / 10;\n mychart1.setOption(option);\n _OverviewActions2.default.getData();\n this.lineDatas();\n }\n }, {\n key: \"render\",\n value: function render() {\n var _this3 = this;\n\n var style = { height: \"30vh\", width: \"calc(40vw - 80px)\", left: \"center\", top: \"center\" };\n return _react2.default.createElement(\n \"div\",\n null,\n _react2.default.createElement(\n \"div\",\n { className: \"selectItemContainer\" },\n _react2.default.createElement(\n \"div\",\n { className: \"selectChart multiSelect\" },\n _react2.default.createElement(_reactSimpleMultiSelect2.default, {\n title: \"Select Chart\",\n itemList: this.state.itemList,\n selectedItemList: this.state.selectedItemList,\n changeList: this.changeList,\n isObjectArray: true\n })\n ),\n _react2.default.createElement(\n \"div\",\n { className: \"select-part\" },\n _react2.default.createElement(\n \"select\",\n { onChange: this.selected.bind(this), value: this.state.step },\n _react2.default.createElement(\n \"option\",\n { value: \"10\" },\n \"Last 10 minutes\"\n ),\n _react2.default.createElement(\n \"option\",\n { value: \"20\" },\n \"Last 20 minutes\"\n ),\n _react2.default.createElement(\n \"option\",\n { value: \"30\" },\n \"Last 30 minutes\"\n )\n )\n )\n ),\n _react2.default.createElement(\n \"div\",\n { className: \"overviewGraphContainerParent\" },\n _react2.default.createElement(\n \"div\",\n { className: \"overviewGraphContainer\" },\n _react2.default.createElement(\n \"div\",\n { className: this.state.checkStatus[\"cpuLoad\"] ? 'overviewChart' : 'display-none' },\n _react2.default.createElement(\"div\", { ref: \"cpuLoad\", style: style })\n ),\n Object.keys(this.state.checkStatus).map(function (key, index) {\n if (key == 'cpuLoad') {\n return null;\n }\n return _react2.default.createElement(\n \"div\",\n { className: _this3.state.checkStatus[key] ? 'overviewChart' : 'display-none', key: index },\n _react2.default.createElement(\"div\", { ref: key, style: style })\n );\n })\n )\n )\n );\n }\n }]);\n\n return EchartPart;\n}(_react2.default.Component);\n\nexports.default = EchartPart;\n\n//# sourceURL=webpack:///./overview/EchartPart.jsx?"); /***/ }), diff --git a/presto-main/src/main/resources/webapp/src/overview/EchartPart.jsx b/presto-main/src/main/resources/webapp/src/overview/EchartPart.jsx index 882802d92c34580a78df2db1a59e4a991576a365..948f6c7c22007c6f165f7f1dd4f7667f5ae72e4d 100644 --- a/presto-main/src/main/resources/webapp/src/overview/EchartPart.jsx +++ b/presto-main/src/main/resources/webapp/src/overview/EchartPart.jsx @@ -37,13 +37,10 @@ class EchartPart extends React.Component{ checkFive:true, checkSix:true, checkSeven:true, - heatMapChart: true, - cpuLoad: true, - heatMapMemoryChart: true + checkEight:true, + cpuLoad: true }, itemList: [ - {key: "Cluster CPU Usage", value: "heatMapChart"}, - {key: "Cluster Free Memory", value: "heatMapMemoryChart"}, {key: "Avg Cluster CPU Usage", value: "cpuLoad"}, {key: "Used Query Memory", value: "checkOne"}, {key: "Running Queries", value: "checkTwo"}, @@ -51,11 +48,10 @@ class EchartPart extends React.Component{ {key: "Blocked Queries", value: "checkFour"}, {key: "Active Workers", value: "checkFive"}, {key: "Avg Running Tasks", value: "checkSix"}, - {key: "Avg CPU cycles per worker", value: "checkSeven"} - ], + {key: "Avg CPU cycles per worker", value: "checkSeven"}, + {key: "Input Total Bytes", value: "checkEight"} + ], selectedItemList: [ - {key: "Cluster CPU Usage", value: "heatMapChart"}, - {key: "Cluster Free Memory", value: "heatMapMemoryChart"}, {key: "Avg Cluster CPU Usage", value: "cpuLoad"}, {key: "Used Query Memory", value: "checkOne"}, {key: "Running Queries", value: "checkTwo"}, @@ -63,14 +59,13 @@ class EchartPart extends React.Component{ {key: "Blocked Queries", value: "checkFour"}, {key: "Active Workers", value: "checkFive"}, {key: "Avg Running Tasks", value: "checkSix"}, - {key: "Avg CPU cycles per worker", value: "checkSeven"} - ], - chartName:['Used Query Memory', 'Running Queries', 'Queued Queries', 'Blocked Queries', 'Active Workers', 'Avg Running Tasks', 'Avg CPU cycles per worker'], + {key: "Avg CPU cycles per worker", value: "checkSeven"}, + {key: "total Input Bytes", value: "checkEight"} + ], + chartName:['Used Query Memory', 'Running Queries', 'Queued Queries', 'Blocked Queries', 'Active Workers', 'Avg Running Tasks', 'Avg CPU cycles per worker','total Input Bytes'], step:10, timer:null, chartCpu:[], - heatMapChart: [], - heatMapMemoryChart: [], chart1:[], chart2:[], chart3:[], @@ -78,16 +73,17 @@ class EchartPart extends React.Component{ chart5:[], chart6:[], chart7:[], + chart8:[], chartRef:null, lastRow:null, lastByte:null, lastWorker:null, memoryInit:false, - unitArr:['bytes','quantity','quantity','quantity','quantity','quantity','quantity'], + unitArr:['bytes','quantity','quantity','quantity','quantity','quantity','quantity','bytes'], lastRefresh: null }; this.state.chartRef = Object.keys(this.state.checkStatus), - this._onChange=this._onChange.bind(this); + this._onChange=this._onChange.bind(this); this.changeList = this.changeList.bind(this); this.resize = this.resize.bind(this); } @@ -151,209 +147,10 @@ class EchartPart extends React.Component{ _onChange(data){ if(data.requestNum%2===0){ if(!this.state.memoryInit && data.memoryData){ - // let cpuChart=echarts.init(this.refs.cpuLoad); - // let option=cpuChart.getOption(); - // let memoryInitData=[]; - // let cpuSeries={}; - // let index = 0; - // Object.keys(data.memoryData).map(key=>{ - // let op = Object.assign({}, option.series[index]); - // index++; - // op.name = key.slice(0, key.indexOf(" ")); - // let currentCpuData = [...this.delete(this.state.chartCpu), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]]; - // op.data = this.state.step === 10 ? currentCpuData.slice(1200) : this.state.step === 20 ? currentCpuData.slice(600) : currentCpuData; - // op.areaStyle = { - // shadowBlur: 10, - // opacity: 0.1 - // }; - // op.type = 'line'; - // op.showSymbol = false; - // memoryInitData.push(op); - // cpuSeries[key]= currentCpuData; - // }) - // option.series=memoryInitData; - // option.yAxis = {max: 100, min: 0, type: "value"}; - // cpuChart.setOption(option); - - let heatMapChart = echarts.init(this.refs.heatMapChart, "royal"); - heatMapChart.setOption({ - animation: false, - title: { - text: 'Cluster CPU Usage', - left: 'center', - textStyle: { - color: "#767676", - fontSize: 16 - } - }, - tooltip:{ - trigger:'item', - formatter: function (params, t, cb) { - return params.name + " : " + params.value+"%"; - } - }, - series: [{ - type: 'treemap', - data: this.state.heatMapChart - }] - }) - let heatMapMemoryChart = echarts.init(this.refs.heatMapMemoryChart, "royal"); - heatMapMemoryChart.setOption({ - animation: false, - title: { - text: 'Cluster Free Memory ', - left: 'center', - textStyle: { - color: "#767676", - fontSize: 16 - } - }, - tooltip:{ - trigger:'item', - formatter: function (params, t, cb) { - return params.name + " : " + formatDataSizeBytes(params.value); - } - }, - series: [{ - type: 'treemap', - data: this.state.heatMapMemoryChart - }] - }) - this.setState({ memoryInit:true }) } - // else{ - // let dataCpu=this.state.chartCpu; - // let mychart1=echarts.init(this.refs.cpuLoad); - // let option=mychart1.getOption(); - // let memoryInitData=option.series; - // Object.keys(data.memoryData).map(key=>{ - // let dataCpuElement = dataCpu[key]; - // if (_.isUndefined(dataCpuElement)) { - // let op = Object.assign({}, option.series[index]); - // op.name = key.slice(0, key.indexOf(" ")); - // op.areaStyle = { - // shadowBlur: 10, - // opacity: 0.1 - // }; - // op.type = 'line'; - // dataCpu[key] = [...this.delete(dataCpuElement), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]]; - // op.data = dataCpu[key]; - // memoryInitData.push(op); - // } - // else { - // dataCpu[key] = [...this.delete(dataCpuElement), [new Date().format('yyyy-MM-dd hh:mm:ss'), (data.memoryData[key].processCpuLoad * 100).toFixed(2)]]; - // } - // for(let i=0,len=memoryInitData.length;i { - let id = data.memoryData[key].id; - let name = key; - let index = _.findIndex(heatMapData, {id: id}); - let newDataPoint = Number((data.memoryData[key].systemCpuLoad * 100).toFixed(2)); - if (index == -1) { - let newData = {}; - newData.id = id; - newData.name = name; - newData.value = newDataPoint - newData.dataset = [newDataPoint]; - newData.children = []; - heatMapData.push(newData); - } - else { - let entry = heatMapData[index]; - let dataset = entry.dataset; - if (dataset.length >= 600) { - dataset = dataset.splice(600 - 1, dataset.length - 600 - 1); - } - dataset = [...dataset, newDataPoint] - entry.dataset = dataset; - let sum = 0; - for (let i = 0; i < dataset.length; i++) { - sum += dataset[i]; - } - entry.value = Number((sum / dataset.length).toFixed(2)); - } - }); - - let heatMapDataSort = bubbleSort(heatMapData); - this.state.heatMapChart = heatMapDataSort.slice(0,10); - let heatMapChart = echarts.init(this.refs.heatMapChart, "royal"); - let heatMapChartOption = heatMapChart.getOption(); - heatMapChartOption.series = [{ - type: "treemap", - data: heatMapDataSort.slice(0,10), - breadcrumb: { - show: false - } - }]; - heatMapChart.setOption(heatMapChartOption); - - //heatMap memory data - let heatMapMemoryData = this.state.heatMapMemoryChart; - Object.keys(data.memoryData).map(key => { - let id = data.memoryData[key].id; - let name = key; - let index = _.findIndex(heatMapMemoryData, {id: id}); - let newDataPoint = 0; - if (typeof (data.memoryData[key].pools.general) != "undefined"){ - newDataPoint += data.memoryData[key].pools.general.freeBytes; - if (typeof (data.memoryData[key].pools.reserved) != "undefined"){ - newDataPoint += data.memoryData[key].pools.reserved.freeBytes; - } - } - newDataPoint = Number(newDataPoint); - if (index == -1) { - let newData = {}; - newData.id = id; - newData.name = name; - newData.value = newDataPoint - newData.dataset = [newDataPoint]; - newData.children = []; - heatMapMemoryData.push(newData); - } - else { - let entry = heatMapMemoryData[index]; - let dataset = entry.dataset; - if (dataset.length >= 600) { - dataset = dataset.splice(600 - 1, dataset.length - 600 - 1); - } - dataset = [...dataset, newDataPoint] - entry.dataset = dataset; - let sum = 0; - for (let i = 0; i < dataset.length; i++) { - sum += dataset[i]; - } - entry.value = Number((sum / dataset.length).toFixed(2)); - } - }); - let heatMapMemoryDataSort = bubbleSort(heatMapMemoryData); - this.state.heatMapMemoryChart = heatMapMemoryDataSort.slice(0,10); - let heatMapMemoryChart = echarts.init(this.refs.heatMapMemoryChart, "royal"); - let heatMapMemoryChartOption = heatMapMemoryChart.getOption(); - heatMapMemoryChartOption.series = [{ - type: "treemap", - data: heatMapMemoryDataSort.slice(0,10), - breadcrumb: { - show: false - }, - }]; - heatMapMemoryChart.setOption(heatMapMemoryChartOption); - let now = Date.now(); let secondsSinceLastRefresh = this.state.lastRefresh ? (now - this.state.lastRefresh) / 1000.0 : 1; secondsSinceLastRefresh = secondsSinceLastRefresh < 1 ? 1 : secondsSinceLastRefresh; @@ -367,9 +164,9 @@ class EchartPart extends React.Component{ chart5:[...this.delete(this.state.chart5),[new Date().format('yyyy-MM-dd hh:mm:ss'),data.lineData.activeWorkers]], chart6:[...this.delete(this.state.chart6),[new Date().format('yyyy-MM-dd hh:mm:ss'),data.lineData.runningDrivers]], chart7:[...this.delete(this.state.chart7),[new Date().format('yyyy-MM-dd hh:mm:ss'),lastWorker]], + chart8:[...this.delete(this.state.chart8),[new Date().format('yyyy-MM-dd hh:mm:ss'),data.lineData.totalInputBytes + ]], lastWorker:data.lineData.totalCpuTimeSecs, - heatMapChart: this.state.heatMapChart, - heatMapMemoryChart: this.state.heatMapMemoryChart, lastRefresh: now }); if (!this.refs.cpuLoad.className) { @@ -428,6 +225,7 @@ class EchartPart extends React.Component{ chart5:[...arr], chart6:[...arr], chart7:[...arr], + chart8:[...arr], }); let mychart1=echarts.init(this.refs.cpuLoad); mychart1.setOption({ @@ -483,13 +281,13 @@ class EchartPart extends React.Component{ mychart.setOption({ animation: false, title:{ - text:this.state.chartName[i], - left:'center', + text:this.state.chartName[i], + left:'center', textStyle: { color: "#767676", fontSize: 16 - } - }, + } + }, tooltip:{ trigger:'axis' }, @@ -552,7 +350,6 @@ class EchartPart extends React.Component{ let mychart=echarts.init(this.refs[this.state.chartRef[i]]); let option=mychart.getOption(); option.xAxis[0].interval=60*1000*this.state.step/10; - // option.series[0].data=[]; mychart.setOption(option); } } @@ -569,45 +366,39 @@ class EchartPart extends React.Component{ return(
-
- + -
-
- -
-
-
-
-
-
-
-
-
+ />
-
-
+
+
- {Object.keys(this.state.checkStatus).map((key, index) => { - if (key == 'cpuLoad' || key == 'heatMapChart' || key == 'heatMapMemoryChart') { - return null; - } - return ( -
-
-
- ) - })}
+
+
+
+
+
+ {Object.keys(this.state.checkStatus).map((key, index) => { + if (key == 'cpuLoad' ) { + return null; + } + return ( +
+
+
+ ) + })} +
diff --git a/presto-main/src/test/java/io/prestosql/execution/buffer/TestPartitionedOutputBuffer.java b/presto-main/src/test/java/io/prestosql/execution/buffer/TestPartitionedOutputBuffer.java index 76d9a937f09093a7255179913344183c9ff82f59..2fd3573048136c0df06bb408480bf2d7f0c1111f 100644 --- a/presto-main/src/test/java/io/prestosql/execution/buffer/TestPartitionedOutputBuffer.java +++ b/presto-main/src/test/java/io/prestosql/execution/buffer/TestPartitionedOutputBuffer.java @@ -26,6 +26,7 @@ import io.prestosql.snapshot.RecoveryUtils; import io.prestosql.snapshot.SnapshotStateId; import io.prestosql.snapshot.TaskSnapshotManager; import io.prestosql.spi.Page; +import io.prestosql.spi.exchange.ExchangeSinkInstanceHandle; import io.prestosql.spi.snapshot.MarkerPage; import io.prestosql.spi.snapshot.SnapshotTestUtil; import io.prestosql.spi.type.BigintType; @@ -1074,4 +1075,14 @@ public class TestPartitionedOutputBuffer List pages = ImmutableList.builder().add(firstPage).add(otherPages).build(); return createBufferResult(token, pages); } + + @Test + public void testBuffersWithExchangeSink() + { + OutputBuffers buffers = createInitialEmptyOutputBuffers(PARTITIONED); + assertFalse(buffers.getExchangeSinkInstanceHandle().isPresent()); + + buffers.setExchangeSinkInstanceHandle(new ExchangeSinkInstanceHandle() {}); + assertTrue(buffers.getExchangeSinkInstanceHandle().isPresent()); + } } diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 5c767718b7e2747bfe74aca92f6d74d7a9fbda03..f0fb5828ee86a9b3e8d062523f3e6ec255d2be77 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -104,5 +104,9 @@ assertj-core test + + org.weakref + jmxutils + diff --git a/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/ObjectNameGeneratorConfig.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/ObjectNameGeneratorConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..1b6ccd204032bf95d227a59352a9def44b52e1d6 --- /dev/null +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/ObjectNameGeneratorConfig.java @@ -0,0 +1,33 @@ +/* + * 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 io.prestosql.plugin.base.jmx; + +import io.airlift.configuration.Config; + +public class ObjectNameGeneratorConfig +{ + private String domainBase; + + public String getDomainBase() + { + return domainBase; + } + + @Config("jmx.base-name") + public ObjectNameGeneratorConfig setDomainBase(String domainBase) + { + this.domainBase = domainBase; + return this; + } +} diff --git a/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/PrefixObjectNameGeneratorModule.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/PrefixObjectNameGeneratorModule.java new file mode 100644 index 0000000000000000000000000000000000000000..f9d16c0e9c067958af6fe1a49357e63ad88e88cb --- /dev/null +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/jmx/PrefixObjectNameGeneratorModule.java @@ -0,0 +1,82 @@ +/* + * 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 io.prestosql.plugin.base.jmx; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.weakref.jmx.ObjectNameBuilder; +import org.weakref.jmx.ObjectNameGenerator; + +import java.util.Map; + +import static com.google.common.base.MoreObjects.firstNonNull; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static java.util.Objects.requireNonNull; + +public class PrefixObjectNameGeneratorModule + implements Module +{ + private final String packageName; + private final String defaultDomainBase; + + public PrefixObjectNameGeneratorModule(String packageName, String defaultDomainBase) + { + this.packageName = requireNonNull(packageName, "packageName is null"); + this.defaultDomainBase = requireNonNull(defaultDomainBase, "defaultDomainBase is null"); + } + + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(ObjectNameGeneratorConfig.class); + } + + @Provides + public ObjectNameGenerator createPrefixObjectNameGenerator(ObjectNameGeneratorConfig config) + { + String domainBase = firstNonNull(config.getDomainBase(), defaultDomainBase); + return new PrefixObjectNameGenerator(packageName, domainBase); + } + + public static final class PrefixObjectNameGenerator + implements ObjectNameGenerator + { + private final String packageName; + private final String domainBase; + + public PrefixObjectNameGenerator(String packageName, String domainBase) + { + this.packageName = requireNonNull(packageName, "packageName is null"); + this.domainBase = requireNonNull(domainBase, "domainBase is null"); + } + + @Override + public String generatedNameOf(Class type, Map properties) + { + return new ObjectNameBuilder(toDomain(type)) + .withProperties(properties) + .build(); + } + + private String toDomain(Class type) + { + String domain = type.getPackage().getName(); + if (domain.startsWith(packageName)) { + domain = domainBase + domain.substring(packageName.length()); + } + return domain; + } + } +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/Plugin.java b/presto-spi/src/main/java/io/prestosql/spi/Plugin.java index d9d8c69f3826d1d1200b9d0491c88c8c090bbb26..dd1809cedc60dbfd896bcd7868de400db90910d3 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/Plugin.java +++ b/presto-spi/src/main/java/io/prestosql/spi/Plugin.java @@ -17,6 +17,7 @@ import io.prestosql.spi.block.BlockEncoding; import io.prestosql.spi.connector.ConnectorFactory; import io.prestosql.spi.cube.CubeProvider; import io.prestosql.spi.eventlistener.EventListenerFactory; +import io.prestosql.spi.exchange.ExchangeManagerFactory; import io.prestosql.spi.failuredetector.FailureRetryFactory; import io.prestosql.spi.filesystem.HetuFileSystemClientFactory; import io.prestosql.spi.function.FunctionNamespaceManagerFactory; @@ -149,19 +150,28 @@ public interface Plugin } default void setExternalFunctionsDir(File externalFuncsDir) - {} + { + } default void setMaxFunctionRunningTimeEnable(boolean enable) - {} + { + } default void setMaxFunctionRunningTimeInSec(long time) - {} + { + } default void setFunctionRunningThreadPoolSize(int size) - {} + { + } default Iterable getFunctionNamespaceManagerFactories() { return emptyList(); } + + default Iterable getExchangeManagerFactories() + { + return emptyList(); + } } diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/Exchange.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/Exchange.java new file mode 100644 index 0000000000000000000000000000000000000000..8e21cc0bf1deba370850c1cd6114b2a3ca62b836 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/Exchange.java @@ -0,0 +1,50 @@ +/* + * 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 io.prestosql.spi.exchange; + +import java.io.Closeable; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public interface Exchange extends Closeable { + /** + * Add a new sink + * @param taskPartitionId unique partition written to a sink + * @return {@link ExchangeSinkHandle} associated with the taskPartitionId + */ + ExchangeSinkHandle addSink(int taskPartitionId); + + /** + * Called when no more sinks will be added with {@link #addSink(int)} + */ + void noMoreSinks(); + + /** + * Registers a sink instance for a task attempt. + * + * @param sinkHandle - handle returned by {@link #addSink(int)} + * @param taskAttemptId - attempt id (how many times attempted) + * @return ExchangeSinkInstanceHandle to be sent to a worker that is needed to create an {@link ExchangeSink} instance + * with {@link ExchangeManager#createSink(ExchangeSinkInstanceHandle, boolean)} + */ + ExchangeSinkInstanceHandle instantiateSink(ExchangeSinkHandle sinkHandle, int taskAttemptId); + + void sinkFinished(ExchangeSinkInstanceHandle handle); + + CompletableFuture> getSourceHandles(); + + ExchangeSourceSplitter split(ExchangeSourceHandle handle, long targetSizeInBytes); + + ExchangeSourceStatistics getExchangeSourceStatistics(ExchangeSourceHandle handle); +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeContext.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeContext.java new file mode 100644 index 0000000000000000000000000000000000000000..2ce464fb8b97bca25b81dfa718a1631179895b21 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeContext.java @@ -0,0 +1,34 @@ +/* + * 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 io.prestosql.spi.exchange; + +import io.prestosql.spi.QueryId; + +public class ExchangeContext { + private final QueryId queryId; + private final ExchangeId exchangeId; + + public ExchangeContext(QueryId queryId, ExchangeId exchangeId) { + this.queryId = queryId; + this.exchangeId = exchangeId; + } + + public QueryId getQueryId() { + return queryId; + } + + public ExchangeId getExchangeId() { + return exchangeId; + } +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeId.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeId.java new file mode 100644 index 0000000000000000000000000000000000000000..ac4248a71f0a08f3b3bbf50ff3e4558017b2f2e6 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeId.java @@ -0,0 +1,69 @@ +/* + * 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 io.prestosql.spi.exchange; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +import java.util.Objects; +import java.util.regex.Pattern; + +import static java.util.Objects.requireNonNull; +import static java.util.UUID.randomUUID; + +public class ExchangeId { + private static final Pattern ID_PATTERN = Pattern.compile("[a-zA-Z\\d_-]+"); + + private final String id; + + @JsonCreator + public ExchangeId(String id) { + requireNonNull(id, "id is null"); + if (!ID_PATTERN.matcher(id).matches()) { + throw new IllegalArgumentException("Invalid exchange id: " + id); + } + this.id = id; + } + + public static ExchangeId createRandomExchangeId() { + return new ExchangeId(randomUUID().toString()); + } + + @JsonValue + public String getId() { + return id; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExchangeId that = (ExchangeId) o; + return Objects.equals(id, that.id); + } + + @Override + public int hashCode() { + return Objects.hash(id); + } + + @Override + public String toString() { + return id; + } +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManager.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManager.java new file mode 100644 index 0000000000000000000000000000000000000000..b1ddbf42b116fc7f4800568fdf8a341f35e9ee56 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManager.java @@ -0,0 +1,38 @@ +/* + * 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 io.prestosql.spi.exchange; + +import javax.annotation.concurrent.ThreadSafe; +import java.util.List; + +/** + * Service provider interface for an external exchange + * It's used to exchange data between stages + */ +@ThreadSafe +public interface ExchangeManager { + /** + * create an external exchange between a pair of stages + * + * @param context information about the query and stage being executed + * @param outputPartitionCount number of distinct partitions to be created by the exchange + * @return {@link Exchange} instance to be used by coordinator to interact with the external exchange + */ + Exchange createExchange(ExchangeContext context, int outputPartitionCount); + + ExchangeSink createSink(ExchangeSinkInstanceHandle handle, boolean preserveRecordsOrder); + + ExchangeSource createSource(List handles); + +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerFactory.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerFactory.java new file mode 100644 index 0000000000000000000000000000000000000000..b8ded8bf2dc2694c78bd68e376e6aa1678b93098 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerFactory.java @@ -0,0 +1,24 @@ +/* + * 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 io.prestosql.spi.exchange; + +import java.util.Map; + +public interface ExchangeManagerFactory { + String getName(); + + ExchangeManager create(Map config); + + ExchangeManagerHandleResolver getHandleResolver(); +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerHandleResolver.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerHandleResolver.java new file mode 100644 index 0000000000000000000000000000000000000000..1189622221e8c71621858b597bcee51339fb8dba --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeManagerHandleResolver.java @@ -0,0 +1,20 @@ +/* + * 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 io.prestosql.spi.exchange; + +public interface ExchangeManagerHandleResolver { + Class getExchangeSinkInstanceHandleClass(); + + Class getExchangeSourceHandleClass(); +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSink.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSink.java new file mode 100644 index 0000000000000000000000000000000000000000..186921ac62403171b4c0287612ec191bd41349ed --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSink.java @@ -0,0 +1,32 @@ +/* + * 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 io.prestosql.spi.exchange; + +import io.airlift.slice.Slice; + +import java.util.concurrent.CompletableFuture; + +public interface ExchangeSink { + CompletableFuture NOT_BLOCKED = CompletableFuture.completedFuture(null); + + CompletableFuture isBlocked(); + + void add(int partitionId, Slice slice); + + long getMemoryUsage(); + + CompletableFuture finish(); + + CompletableFuture abort(); +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkHandle.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..c4bf218516456703e69ad904c4fdd555f468b49f --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkHandle.java @@ -0,0 +1,17 @@ +/* + * 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 io.prestosql.spi.exchange; + +public interface ExchangeSinkHandle { +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkInstanceHandle.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkInstanceHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..69d20e9d09c50ce33a9942618e5ece8b284181d0 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSinkInstanceHandle.java @@ -0,0 +1,17 @@ +/* + * 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 io.prestosql.spi.exchange; + +public interface ExchangeSinkInstanceHandle { +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSource.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSource.java new file mode 100644 index 0000000000000000000000000000000000000000..72781f08d6d58d7b917690069f0f0b624ee3069d --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSource.java @@ -0,0 +1,38 @@ +/* + * 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 io.prestosql.spi.exchange; + +import io.airlift.slice.Slice; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; +import java.io.Closeable; +import java.util.concurrent.CompletableFuture; + +@ThreadSafe +public interface ExchangeSource extends Closeable { + CompletableFuture NOT_BLOCKED = CompletableFuture.completedFuture(null); + + CompletableFuture isBlocked(); + + boolean isFinished(); + + @Nullable + Slice read(); + + long getMemoryUsage(); + + @Override + void close(); +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceHandle.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceHandle.java new file mode 100644 index 0000000000000000000000000000000000000000..bee1854214253b16bef75fe1ad5a8b7753ce2902 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceHandle.java @@ -0,0 +1,17 @@ +/* + * 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 io.prestosql.spi.exchange; + +public interface ExchangeSourceHandle { +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceSplitter.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceSplitter.java new file mode 100644 index 0000000000000000000000000000000000000000..f9f4e07b8e9cd915b8b680e67b7cabc4689d3217 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceSplitter.java @@ -0,0 +1,17 @@ +/* + * 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 io.prestosql.spi.exchange; + +public class ExchangeSourceSplitter { +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceStatistics.java b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceStatistics.java new file mode 100644 index 0000000000000000000000000000000000000000..4a5da5007152d6213526ae0ebe647d7340a4bc84 --- /dev/null +++ b/presto-spi/src/main/java/io/prestosql/spi/exchange/ExchangeSourceStatistics.java @@ -0,0 +1,26 @@ +/* + * 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 io.prestosql.spi.exchange; + +public class ExchangeSourceStatistics { + private final long sizeInBytes; + + public ExchangeSourceStatistics(long sizeInBytes) { + this.sizeInBytes = sizeInBytes; + } + + public long getSizeInBytes() { + return sizeInBytes; + } +} diff --git a/src/main/resource/license/license-header-alternate-2022-1.txt b/src/main/resource/license/license-header-alternate-2022-1.txt new file mode 100644 index 0000000000000000000000000000000000000000..3c88176961d3e1ccd652e9014b87c4dd8cc2a6b6 --- /dev/null +++ b/src/main/resource/license/license-header-alternate-2022-1.txt @@ -0,0 +1,12 @@ +Copyright (C) 2022-2022. Yijian Cheng. All rights reserved. +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.