├── .gitignore ├── .travis.yml ├── CODE_OF_CONDUCT.md ├── LICENSE ├── README.md ├── athenax-backend ├── pom.xml └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── uber │ │ │ └── athenax │ │ │ └── backend │ │ │ ├── AthenaXServer.java │ │ │ ├── api │ │ │ ├── factories │ │ │ │ ├── ClusterApiServiceFactory.java │ │ │ │ ├── InstancesApiServiceFactory.java │ │ │ │ ├── JobsApiServiceFactory.java │ │ │ │ └── package-info.java │ │ │ └── impl │ │ │ │ ├── ClusterApiServiceImpl.java │ │ │ │ ├── InstancesApiServiceImpl.java │ │ │ │ ├── JobsApiServiceImpl.java │ │ │ │ └── package-info.java │ │ │ ├── package-info.java │ │ │ └── server │ │ │ ├── AthenaXConfiguration.java │ │ │ ├── AthenaXExtraConfigOptions.java │ │ │ ├── InstanceStateUpdateListener.java │ │ │ ├── ServerContext.java │ │ │ ├── WebServer.java │ │ │ ├── jobs │ │ │ ├── HealthCheckReport.java │ │ │ ├── JobManager.java │ │ │ ├── JobStore.java │ │ │ ├── JobWatcherUtil.java │ │ │ ├── LevelDBJobStore.java │ │ │ ├── WatchdogPolicy.java │ │ │ ├── WatchdogPolicyDefault.java │ │ │ └── package-info.java │ │ │ ├── package-info.java │ │ │ └── yarn │ │ │ ├── AthenaXYarnClusterDescriptor.java │ │ │ ├── ClusterInfo.java │ │ │ ├── InstanceInfo.java │ │ │ ├── InstanceManager.java │ │ │ ├── InstanceMetadata.java │ │ │ ├── JobConf.java │ │ │ ├── JobDeployer.java │ │ │ ├── Utils.java │ │ │ ├── YarnClusterConfiguration.java │ │ │ └── package-info.java │ └── resources │ │ ├── .swagger-codegen-ignore │ │ └── athenax-backend-api.yaml │ └── test │ ├── java │ └── com │ │ └── uber │ │ └── athenax │ │ └── backend │ │ └── server │ │ ├── AthenaXConfigurationTest.java │ │ ├── MiniAthenaXCluster.java │ │ ├── jobs │ │ ├── JobWatcherUtilTest.java │ │ └── LevelDBJobStoreTest.java │ │ └── yarn │ │ ├── InstanceManagerITest.java │ │ ├── InstanceManagerTest.java │ │ ├── InstanceMetadataTest.java │ │ ├── JobDeployerITest.java │ │ ├── JobDeployerTest.java │ │ └── JobITestUtil.java │ └── resources │ ├── athenax-conf-test1.yaml │ └── log4j.properties ├── athenax-tests ├── pom.xml └── src │ └── test │ ├── java │ └── com │ │ └── uber │ │ └── athenax │ │ └── tests │ │ ├── ITestUtil.java │ │ └── StartJobITest.java │ └── resources │ └── log4j.properties ├── athenax-vm-api ├── pom.xml └── src │ └── main │ └── java │ └── com │ └── uber │ └── athenax │ └── vm │ └── api │ ├── functions │ ├── AthenaXAggregateFunction.java │ ├── AthenaXScalarFunction.java │ ├── AthenaXTableFunction.java │ └── package-info.java │ └── tables │ ├── AthenaXTableCatalog.java │ ├── AthenaXTableCatalogProvider.java │ ├── AthenaXTableSinkProvider.java │ └── package-info.java ├── athenax-vm-compiler ├── pom.xml └── src │ ├── codegen │ ├── config.fmpp │ ├── data │ │ └── Parser.tdd │ └── includes │ │ └── parserImpls.ftl │ ├── main │ └── java │ │ └── com │ │ └── uber │ │ └── athenax │ │ └── vm │ │ └── compiler │ │ ├── executor │ │ ├── CompilationResult.java │ │ ├── ContainedExecutor.java │ │ ├── JobCompiler.java │ │ ├── JobDescriptor.java │ │ ├── TableSinkProviderRegistry.java │ │ └── package-info.java │ │ ├── parser │ │ ├── SqlCreateFunction.java │ │ ├── UnparseUtil.java │ │ └── package-info.java │ │ └── planner │ │ ├── JobCompilationResult.java │ │ ├── Planner.java │ │ ├── Validator.java │ │ └── package-info.java │ └── test │ ├── java │ └── com │ │ └── uber │ │ └── athenax │ │ └── vm │ │ └── compiler │ │ ├── executor │ │ └── ProcessExecutorTest.java │ │ └── planner │ │ ├── PlannerTest.java │ │ └── ValidatorTest.java │ └── resources │ └── log4j.properties ├── athenax-vm-connectors ├── athenax-vm-connector-common │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── com │ │ │ └── uber │ │ │ └── athenax │ │ │ └── vm │ │ │ └── compiler │ │ │ └── executor │ │ │ ├── MockAppendStreamTableSink.java │ │ │ ├── MockExternalCatalogTable.java │ │ │ ├── MockTableSinkProvider.java │ │ │ ├── MockTableSource.java │ │ │ ├── MockTableSourceFactory.java │ │ │ └── SingleLevelMemoryCatalog.java │ │ └── resources │ │ └── META-INF │ │ └── services │ │ ├── com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider │ │ └── org.apache.flink.table.sources.TableSourceFactory └── athenax-vm-connector-kafka │ ├── pom.xml │ └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── uber │ │ │ └── athenax │ │ │ └── vm │ │ │ └── connectors │ │ │ └── kafka │ │ │ ├── JsonTableSource.java │ │ │ ├── JsonTableSourceFactory.java │ │ │ ├── KafkaConnectorDescriptorValidator.java │ │ │ ├── KafkaJsonConnector.java │ │ │ ├── KafkaUtils.java │ │ │ └── package-info.java │ └── resources │ │ └── META-INF │ │ └── services │ │ ├── com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider │ │ └── org.apache.flink.table.sources.TableSourceFactory │ └── test │ ├── java │ └── com │ │ └── uber │ │ └── athenax │ │ └── vm │ │ └── connectors │ │ └── kafka │ │ ├── EmbeddedZooKeeper.java │ │ ├── FlinkTestUtil.java │ │ ├── KafkaJsonConnectorITest.java │ │ ├── KafkaTestUtil.java │ │ ├── KafkaUtilsTest.java │ │ └── MiniKafkaCluster.java │ └── resources │ └── log4j.properties ├── dev-support ├── checkstyle.xml ├── findbugs-excludes.xml ├── suppressions.xml └── test-patch.py ├── docs ├── README.md ├── design.md ├── examples.md ├── getting_started.md ├── images │ ├── architecture.svg │ ├── athena-logo.png │ ├── athena.svg │ └── favicon.ico ├── index.md ├── requirements.txt └── theme │ ├── 404.html │ ├── __init__.py │ ├── assets │ ├── images │ │ ├── favicon.ico │ │ ├── favicon.png │ │ └── icons │ │ │ ├── bitbucket-670608a71a.svg │ │ │ ├── github-1da075986e.svg │ │ │ └── gitlab-5ad3f9f9e5.svg │ ├── javascripts │ │ ├── application-ae2e7c9801.js │ │ └── modernizr-56ade86843.js │ └── stylesheets │ │ ├── application-60987dc207.css │ │ ├── application-a9490fa5b9.css │ │ └── application-f78e5cb881.palette.css │ ├── base.html │ ├── main.html │ └── partials │ ├── disqus.html │ ├── footer.html │ ├── header.html │ ├── language.html │ ├── nav-item.html │ ├── nav.html │ ├── search.html │ ├── social.html │ ├── source.html │ ├── tabs-item.html │ ├── tabs.html │ ├── toc-item.html │ └── toc.html ├── mkdocs.yml └── pom.xml /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # BlueJ files 8 | *.ctxt 9 | 10 | # Mobile Tools for Java (J2ME) 11 | .mtj.tmp/ 12 | 13 | # Package Files # 14 | *.jar 15 | *.war 16 | *.ear 17 | *.zip 18 | *.tar.gz 19 | *.rar 20 | 21 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 22 | hs_err_pid* 23 | 24 | # Intellij files 25 | *.iml 26 | .idea 27 | 28 | target 29 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | sudo: false 17 | dist: trusty 18 | 19 | cache: 20 | directories: 21 | - $HOME/.m2 22 | 23 | # do not cache our own artifacts 24 | before_cache: 25 | - rm -rf $HOME/.m2/repository/com/uber/athenax 26 | 27 | install: true 28 | 29 | language: java 30 | 31 | matrix: 32 | include: 33 | - jdk: "openjdk8" 34 | 35 | git: 36 | depth: 100 37 | 38 | env: 39 | global: 40 | # Global variable to avoid hanging travis builds when downloading cache archives. 41 | - MALLOC_ARENA_MAX=2 42 | 43 | # We run mvn and monitor its output. If there is no output for the specified number of seconds, we 44 | # print the stack traces of all running Java processes. 45 | script: "./dev-support/test-patch.py" 46 | -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | # Contributor Covenant Code of Conduct 2 | 3 | ## Our Pledge 4 | 5 | In the interest of fostering an open and welcoming environment, we as contributors and maintainers pledge to making participation in our project and our community a harassment-free experience for everyone, regardless of age, body size, disability, ethnicity, gender identity and expression, level of experience, nationality, personal appearance, race, religion, or sexual identity and orientation. 6 | 7 | ## Our Standards 8 | 9 | Examples of behavior that contributes to creating a positive environment include: 10 | 11 | * Using welcoming and inclusive language 12 | * Being respectful of differing viewpoints and experiences 13 | * Gracefully accepting constructive criticism 14 | * Focusing on what is best for the community 15 | * Showing empathy towards other community members 16 | 17 | Examples of unacceptable behavior by participants include: 18 | 19 | * The use of sexualized language or imagery and unwelcome sexual attention or advances 20 | * Trolling, insulting/derogatory comments, and personal or political attacks 21 | * Public or private harassment 22 | * Publishing others' private information, such as a physical or electronic address, without explicit permission 23 | * Other conduct which could reasonably be considered inappropriate in a professional setting 24 | 25 | ## Our Responsibilities 26 | 27 | Project maintainers are responsible for clarifying the standards of acceptable behavior and are expected to take appropriate and fair corrective action in response to any instances of unacceptable behavior. 28 | 29 | Project maintainers have the right and responsibility to remove, edit, or reject comments, commits, code, wiki edits, issues, and other contributions that are not aligned to this Code of Conduct, or to ban temporarily or permanently any contributor for other behaviors that they deem inappropriate, threatening, offensive, or harmful. 30 | 31 | ## Scope 32 | 33 | This Code of Conduct applies both within project spaces and in public spaces when an individual is representing the project or its community. Examples of representing a project or community include using an official project e-mail address, posting via an official social media account, or acting as an appointed representative at an online or offline event. Representation of a project may be further defined and clarified by project maintainers. 34 | 35 | ## Enforcement 36 | 37 | Instances of abusive, harassing, or otherwise unacceptable behavior may be reported by contacting the project team at haohui@uber.com. The project team will review and investigate all complaints, and will respond in a way that it deems appropriate to the circumstances. The project team is obligated to maintain confidentiality with regard to the reporter of an incident. Further details of specific enforcement policies may be posted separately. 38 | 39 | Project maintainers who do not follow or enforce the Code of Conduct in good faith may face temporary or permanent repercussions as determined by other members of the project's leadership. 40 | 41 | ## Attribution 42 | 43 | This Code of Conduct is adapted from the [Contributor Covenant][homepage], version 1.4, available at [http://contributor-covenant.org/version/1/4][version] 44 | 45 | [homepage]: http://contributor-covenant.org 46 | [version]: http://contributor-covenant.org/version/1/4/ 47 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | [![Build Status][ci-img]][ci] [![ReadTheDocs][doc-img]][doc] 2 | 3 | # AthenaX: SQL-based streaming analytics platform at scale 4 | 5 | AthenaX is a streaming analytics platform that enables users to run production-quality, large scale streaming analytics using Structured Query Language (SQL). AthenaX was released and open sourced by [Uber Technologies][ubeross]. It is capable of scaling across hundreds of machines and processing hundreds of billions of real-time events daily. 6 | 7 | See also: 8 | 9 | * AthenaX [documentation][doc] for getting started, operational details, and other information. 10 | * Blog post [Introducing AthenaX, Uber Engineering’s Open Source Streaming Analytics Platform](https://eng.uber.com/athenax/). 11 | 12 | ## License 13 | [Apache 2.0 License](./LICENSE). 14 | 15 | [doc-img]: https://readthedocs.org/projects/athenax/badge/?version=latest 16 | [doc]: http://athenax.readthedocs.org/en/latest/ 17 | [ci-img]: https://travis-ci.org/jaegertracing/jaeger.svg?branch=master 18 | [ci]: https://travis-ci.org/uber/AthenaX 19 | [ubeross]: http://uber.github.io 20 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/AthenaXServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend; 20 | 21 | import com.uber.athenax.backend.server.AthenaXConfiguration; 22 | import com.uber.athenax.backend.server.ServerContext; 23 | import com.uber.athenax.backend.server.WebServer; 24 | import org.apache.commons.cli.CommandLine; 25 | import org.apache.commons.cli.CommandLineParser; 26 | import org.apache.commons.cli.DefaultParser; 27 | import org.apache.commons.cli.HelpFormatter; 28 | import org.apache.commons.cli.Options; 29 | 30 | import java.io.IOException; 31 | import java.net.URI; 32 | import java.nio.file.Paths; 33 | 34 | public class AthenaXServer { 35 | private static final Options CLI_OPTIONS = new Options() 36 | .addOption(null, "conf", true, "The configuration file"); 37 | 38 | private void start(AthenaXConfiguration conf) throws Exception { 39 | ServerContext.INSTANCE.initialize(conf); 40 | ServerContext.INSTANCE.start(); 41 | try (WebServer server = new WebServer(URI.create(conf.masterUri()))) { 42 | server.start(); 43 | Thread.currentThread().join(); 44 | } 45 | } 46 | 47 | public static void main(String[] args) throws Exception { 48 | CommandLineParser parser = new DefaultParser(); 49 | CommandLine line = parser.parse(CLI_OPTIONS, args); 50 | if (!line.hasOption("conf")) { 51 | System.err.println("No configuration file is specified"); 52 | HelpFormatter formatter = new HelpFormatter(); 53 | formatter.printHelp("athenax-server", CLI_OPTIONS); 54 | System.exit(1); 55 | } 56 | 57 | try { 58 | String confFile = line.getOptionValue("conf"); 59 | AthenaXConfiguration conf = AthenaXConfiguration.load(Paths.get(confFile).toFile()); 60 | new AthenaXServer().start(conf); 61 | } catch (IOException | ClassNotFoundException e) { 62 | System.err.println("Failed to parse configuration."); 63 | throw e; 64 | } 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/factories/ClusterApiServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.factories; 20 | 21 | import com.uber.athenax.backend.api.ClusterApiService; 22 | import com.uber.athenax.backend.api.impl.ClusterApiServiceImpl; 23 | import com.uber.athenax.backend.server.ServerContext; 24 | 25 | @javax.annotation.Generated( 26 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 27 | date = "2017-09-19T15:16:54.206-07:00") 28 | public final class ClusterApiServiceFactory { 29 | private static final ClusterApiService SERVICE = new ClusterApiServiceImpl(ServerContext.INSTANCE); 30 | 31 | private ClusterApiServiceFactory() { 32 | } 33 | 34 | public static ClusterApiService getClusterApi() { 35 | return SERVICE; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/factories/InstancesApiServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.factories; 20 | 21 | import com.uber.athenax.backend.api.InstancesApiService; 22 | import com.uber.athenax.backend.api.impl.InstancesApiServiceImpl; 23 | import com.uber.athenax.backend.server.ServerContext; 24 | 25 | @javax.annotation.Generated( 26 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 27 | date = "2017-09-19T15:16:54.206-07:00") 28 | public final class InstancesApiServiceFactory { 29 | private static final InstancesApiService SERVICE = new InstancesApiServiceImpl(ServerContext.INSTANCE); 30 | 31 | private InstancesApiServiceFactory() { 32 | } 33 | 34 | public static InstancesApiService getInstancesApi() { 35 | return SERVICE; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/factories/JobsApiServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.factories; 20 | 21 | import com.uber.athenax.backend.api.JobsApiService; 22 | import com.uber.athenax.backend.api.impl.JobsApiServiceImpl; 23 | import com.uber.athenax.backend.server.ServerContext; 24 | 25 | @javax.annotation.Generated( 26 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 27 | date = "2017-09-22T14:08:01.325-07:00") 28 | public final class JobsApiServiceFactory { 29 | private static final JobsApiService SERVICE = new JobsApiServiceImpl(ServerContext.INSTANCE); 30 | 31 | private JobsApiServiceFactory() { 32 | } 33 | 34 | public static JobsApiService getJobsApi() { 35 | return SERVICE; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/factories/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * Factory classes that create the corresponding implementation. 21 | */ 22 | package com.uber.athenax.backend.api.factories; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/impl/ClusterApiServiceImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.impl; 20 | 21 | import com.uber.athenax.backend.api.ClusterApiService; 22 | import com.uber.athenax.backend.api.ClusterInfo; 23 | import com.uber.athenax.backend.api.NotFoundException; 24 | import com.uber.athenax.backend.server.ServerContext; 25 | 26 | import javax.ws.rs.core.Response; 27 | import javax.ws.rs.core.SecurityContext; 28 | 29 | @javax.annotation.Generated( 30 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 31 | date = "2017-09-15T14:49:25.985-07:00") 32 | public class ClusterApiServiceImpl extends ClusterApiService { 33 | private final ServerContext ctx; 34 | 35 | public ClusterApiServiceImpl(ServerContext ctx) { 36 | this.ctx = ctx; 37 | } 38 | 39 | @Override 40 | public Response getClusterInfo(SecurityContext securityContext) throws NotFoundException { 41 | ClusterInfo info = new ClusterInfo().startedOn(ctx.startTime()); 42 | return Response.ok().entity(info).build(); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/impl/InstancesApiServiceImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.impl; 20 | 21 | import com.uber.athenax.backend.api.InstanceState; 22 | import com.uber.athenax.backend.api.InstanceStatus; 23 | import com.uber.athenax.backend.api.InstancesApiService; 24 | import com.uber.athenax.backend.api.NotFoundException; 25 | import com.uber.athenax.backend.server.ServerContext; 26 | import org.apache.hadoop.yarn.exceptions.YarnException; 27 | import org.apache.http.HttpStatus; 28 | 29 | import javax.ws.rs.core.Response; 30 | import javax.ws.rs.core.SecurityContext; 31 | import java.io.IOException; 32 | import java.util.UUID; 33 | 34 | @javax.annotation.Generated( 35 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 36 | date = "2017-09-19T15:16:54.206-07:00") 37 | public class InstancesApiServiceImpl extends InstancesApiService { 38 | private final ServerContext ctx; 39 | 40 | public InstancesApiServiceImpl(ServerContext ctx) { 41 | this.ctx = ctx; 42 | } 43 | 44 | @Override 45 | public Response changeInstanceState( 46 | UUID instanceUUID, 47 | InstanceState state, 48 | SecurityContext securityContext) throws NotFoundException { 49 | try { 50 | ctx.instanceManager().changeState(instanceUUID, state); 51 | } catch (YarnException | IOException e) { 52 | return Response.serverError().entity("Failed to kill the applications " + e).build(); 53 | } 54 | return Response.ok().build(); 55 | } 56 | 57 | @Override 58 | public Response getInstanceInfo(UUID instanceUUID, SecurityContext securityContext) throws NotFoundException { 59 | InstanceStatus stat = ctx.instanceManager().getInstanceStatus(instanceUUID); 60 | if (stat != null) { 61 | return Response.ok().entity(stat).build(); 62 | } else { 63 | throw new NotFoundException(HttpStatus.SC_NOT_FOUND, "Instance not found"); 64 | } 65 | } 66 | 67 | @Override 68 | public Response getInstanceState(UUID instanceUUID, SecurityContext securityContext) throws NotFoundException { 69 | InstanceState state = ctx.instanceManager().getInstanceState(instanceUUID); 70 | if (state != null) { 71 | return Response.ok().entity(state).build(); 72 | } else { 73 | throw new NotFoundException(HttpStatus.SC_NOT_FOUND, "Instance not found"); 74 | } 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/impl/JobsApiServiceImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.api.impl; 20 | 21 | import com.uber.athenax.backend.api.JobDefinition; 22 | import com.uber.athenax.backend.api.JobsApiService; 23 | import com.uber.athenax.backend.api.NotFoundException; 24 | import com.uber.athenax.backend.server.ServerContext; 25 | import org.apache.http.HttpStatus; 26 | 27 | import javax.ws.rs.core.Response; 28 | import javax.ws.rs.core.SecurityContext; 29 | import java.io.IOException; 30 | import java.util.Collections; 31 | import java.util.UUID; 32 | 33 | @javax.annotation.Generated( 34 | value = "io.swagger.codegen.languages.JavaJerseyServerCodegen", 35 | date = "2017-09-22T14:43:25.370-07:00") 36 | public class JobsApiServiceImpl extends JobsApiService { 37 | public static final int INVALID_REQUEST = HttpStatus.SC_INTERNAL_SERVER_ERROR; 38 | private final ServerContext ctx; 39 | 40 | public JobsApiServiceImpl(ServerContext ctx) { 41 | this.ctx = ctx; 42 | } 43 | 44 | @Override 45 | public Response allocateNewJob(SecurityContext securityContext) throws NotFoundException { 46 | return Response.ok().entity( 47 | Collections.singletonMap("job-uuid", ctx.jobManager().newJobUUID()) 48 | ).build(); 49 | } 50 | 51 | @Override 52 | public Response getJob(UUID jobUUID, SecurityContext securityContext) throws NotFoundException { 53 | try { 54 | JobDefinition job = ctx.jobManager().getJob(jobUUID); 55 | if (job == null) { 56 | return Response.status(Response.Status.NOT_FOUND).build(); 57 | } else { 58 | return Response.ok(job).build(); 59 | } 60 | } catch (IOException e) { 61 | throw new NotFoundException(INVALID_REQUEST, e.getMessage()); 62 | } 63 | } 64 | 65 | @Override 66 | public Response listJob(SecurityContext securityContext) throws NotFoundException { 67 | try { 68 | return Response.ok(ctx.jobManager().listJobs()).build(); 69 | } catch (IOException e) { 70 | throw new NotFoundException(INVALID_REQUEST, e.getMessage()); 71 | } 72 | } 73 | 74 | @Override 75 | public Response removeJob(UUID jobUUID, SecurityContext securityContext) throws NotFoundException { 76 | try { 77 | ctx.jobManager().removeJob(jobUUID); 78 | } catch (IOException e) { 79 | throw new NotFoundException(INVALID_REQUEST, e.getMessage()); 80 | } 81 | return Response.ok().build(); 82 | } 83 | 84 | @Override 85 | public Response updateJob( 86 | UUID jobUUID, 87 | JobDefinition body, 88 | SecurityContext securityContext) throws NotFoundException { 89 | try { 90 | ctx.jobManager().updateJob(jobUUID, body); 91 | } catch (IOException e) { 92 | throw new NotFoundException(INVALID_REQUEST, e.getMessage()); 93 | } 94 | return Response.ok().build(); 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/api/impl/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * Glue layer of between REST and actual implementation. 21 | */ 22 | package com.uber.athenax.backend.api.impl; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * The REST Backend for AthenaX. 21 | */ 22 | package com.uber.athenax.backend; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/AthenaXExtraConfigOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | 23 | import static org.apache.flink.configuration.ConfigOptions.key; 24 | 25 | public final class AthenaXExtraConfigOptions { 26 | 27 | /** 28 | * The directory that the LevelDB JobStore uses to persist information. 29 | */ 30 | public static final ConfigOption JOBSTORE_LEVELDB_FILE = 31 | key("jobstore.leveldb.file").noDefaultValue(); 32 | 33 | /** 34 | * The interval that the instance manager will rescan all running AthenaX application. 35 | */ 36 | public static final ConfigOption INSTANCE_MANAGER_RESCAN_INTERVAL = 37 | key("instancemanager.rescan.interval").defaultValue(120 * 1000L); 38 | 39 | private AthenaXExtraConfigOptions() { 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/InstanceStateUpdateListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server; 20 | 21 | import com.uber.athenax.backend.server.yarn.InstanceInfo; 22 | 23 | import java.util.UUID; 24 | import java.util.concurrent.ConcurrentHashMap; 25 | 26 | public interface InstanceStateUpdateListener { 27 | void onUpdatedInstances(ConcurrentHashMap instances); 28 | } 29 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/ServerContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server; 20 | 21 | import com.uber.athenax.backend.server.jobs.JobManager; 22 | import com.uber.athenax.backend.server.jobs.JobStore; 23 | import com.uber.athenax.backend.server.jobs.WatchdogPolicy; 24 | import com.uber.athenax.backend.server.yarn.InstanceManager; 25 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalogProvider; 26 | 27 | import java.io.IOException; 28 | import java.util.concurrent.Executors; 29 | import java.util.concurrent.ScheduledExecutorService; 30 | 31 | import static org.apache.flink.util.InstantiationUtil.instantiate; 32 | 33 | public final class ServerContext { 34 | public static final ServerContext INSTANCE = new ServerContext(); 35 | 36 | private final long startTime; 37 | private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); 38 | private InstanceManager instanceManager; 39 | private JobStore jobStore; 40 | private JobManager jobManager; 41 | private WatchdogPolicy watchdogPolicy; 42 | private AthenaXTableCatalogProvider catalogs; 43 | private AthenaXConfiguration conf; 44 | 45 | private ServerContext() { 46 | this.startTime = System.currentTimeMillis(); 47 | } 48 | 49 | public void initialize(AthenaXConfiguration conf) throws ClassNotFoundException, IOException { 50 | this.conf = conf; 51 | this.jobStore = (JobStore) instantiate(Class.forName(conf.jobStoreImpl())); 52 | this.catalogs = (AthenaXTableCatalogProvider) instantiate(Class.forName(conf.catalogProvider())); 53 | this.jobManager = new JobManager(jobStore, catalogs); 54 | this.instanceManager = InstanceManager.create(conf, jobManager, executor); 55 | this.watchdogPolicy = (WatchdogPolicy) instantiate(Class.forName(conf.watchdogPolicyImpl())); 56 | } 57 | 58 | public WatchdogPolicy watchdogPolicy() { 59 | return watchdogPolicy; 60 | } 61 | 62 | public void start() throws IOException { 63 | jobStore.open(conf); 64 | instanceManager.start(); 65 | } 66 | 67 | public long startTime() { 68 | return startTime; 69 | } 70 | 71 | public InstanceManager instanceManager() { 72 | return instanceManager; 73 | } 74 | 75 | public JobManager jobManager() { 76 | return jobManager; 77 | } 78 | 79 | public AthenaXTableCatalogProvider catalogs() { 80 | return catalogs; 81 | } 82 | 83 | public ScheduledExecutorService executor() { 84 | return executor; 85 | } 86 | 87 | } 88 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/WebServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server; 20 | 21 | import com.sun.jersey.api.container.grizzly2.GrizzlyServerFactory; 22 | import com.sun.jersey.api.core.PackagesResourceConfig; 23 | import com.sun.jersey.spi.container.servlet.ServletContainer; 24 | import com.uber.athenax.backend.api.ClusterApi; 25 | import org.glassfish.grizzly.http.server.HttpHandler; 26 | import org.glassfish.grizzly.http.server.HttpServer; 27 | import org.glassfish.grizzly.http.server.Request; 28 | import org.glassfish.grizzly.http.server.Response; 29 | import org.glassfish.grizzly.http.util.HttpStatus; 30 | import org.glassfish.grizzly.servlet.ServletRegistration; 31 | import org.glassfish.grizzly.servlet.WebappContext; 32 | 33 | import java.io.IOException; 34 | import java.net.URI; 35 | import java.util.StringJoiner; 36 | 37 | public class WebServer implements AutoCloseable { 38 | public static final String BASE_PATH = "/ws/v1"; 39 | 40 | private static final String[] PACKAGES = new String[] { 41 | ClusterApi.class.getPackage().getName(), 42 | "com.fasterxml.jackson.jaxrs.json" 43 | }; 44 | 45 | private final HttpServer server; 46 | 47 | public WebServer(URI endpoint) throws IOException { 48 | this.server = GrizzlyServerFactory.createHttpServer(endpoint, new HttpHandler() { 49 | 50 | @Override 51 | public void service(Request rqst, Response rspns) throws Exception { 52 | rspns.setStatus(HttpStatus.NOT_FOUND_404.getStatusCode(), "Not found"); 53 | rspns.getWriter().write("404: not found"); 54 | } 55 | }); 56 | 57 | WebappContext context = new WebappContext("WebappContext", BASE_PATH); 58 | ServletRegistration registration = context.addServlet("ServletContainer", ServletContainer.class); 59 | registration.setInitParameter(ServletContainer.RESOURCE_CONFIG_CLASS, 60 | PackagesResourceConfig.class.getName()); 61 | 62 | StringJoiner sj = new StringJoiner(","); 63 | for (String s : PACKAGES) { 64 | sj.add(s); 65 | } 66 | 67 | registration.setInitParameter(PackagesResourceConfig.PROPERTY_PACKAGES, sj.toString()); 68 | registration.addMapping(BASE_PATH); 69 | context.deploy(server); 70 | } 71 | 72 | public void start() throws IOException { 73 | server.start(); 74 | } 75 | 76 | @Override 77 | public void close() throws Exception { 78 | server.stop(); 79 | } 80 | 81 | public int port() { 82 | return server.getListeners().iterator().next().getPort(); 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/HealthCheckReport.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | import com.uber.athenax.backend.api.ExtendedJobDefinition; 22 | import com.uber.athenax.backend.api.JobDefinitionDesiredstate; 23 | import com.uber.athenax.backend.server.yarn.InstanceInfo; 24 | 25 | import java.util.ArrayList; 26 | import java.util.HashMap; 27 | import java.util.List; 28 | import java.util.Map; 29 | 30 | public class HealthCheckReport { 31 | private final List spuriousInstances = new ArrayList(); 32 | private final List instancesWithDifferentParameters = new ArrayList<>(); 33 | private final Map> instancesToStart = new HashMap<>(); 34 | 35 | public List spuriousInstances() { 36 | return spuriousInstances; 37 | } 38 | 39 | public List instancesWithDifferentParameters() { 40 | return instancesWithDifferentParameters; 41 | } 42 | 43 | public Map> instancesToStart() { 44 | return instancesToStart; 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/JobManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | import com.uber.athenax.backend.api.ExtendedJobDefinition; 22 | import com.uber.athenax.backend.api.JobDefinition; 23 | import com.uber.athenax.backend.api.JobDefinitionDesiredstate; 24 | import com.uber.athenax.backend.server.InstanceStateUpdateListener; 25 | import com.uber.athenax.backend.server.ServerContext; 26 | import com.uber.athenax.backend.server.yarn.InstanceInfo; 27 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalog; 28 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalogProvider; 29 | import com.uber.athenax.vm.compiler.planner.JobCompilationResult; 30 | import com.uber.athenax.vm.compiler.planner.Planner; 31 | import org.slf4j.Logger; 32 | import org.slf4j.LoggerFactory; 33 | 34 | import java.io.IOException; 35 | import java.util.HashMap; 36 | import java.util.List; 37 | import java.util.Map; 38 | import java.util.UUID; 39 | import java.util.concurrent.ConcurrentHashMap; 40 | 41 | public class JobManager implements InstanceStateUpdateListener { 42 | private static final Logger LOG = LoggerFactory.getLogger(JobManager.class); 43 | private final JobStore jobStore; 44 | private final AthenaXTableCatalogProvider catalogProvider; 45 | 46 | public JobManager(JobStore jobStore, AthenaXTableCatalogProvider catalogProvider) { 47 | this.jobStore = jobStore; 48 | this.catalogProvider = catalogProvider; 49 | } 50 | 51 | public UUID newJobUUID() { 52 | return UUID.randomUUID(); 53 | } 54 | 55 | public void updateJob(UUID uuid, JobDefinition definition) throws IOException { 56 | jobStore.updateJob(uuid, definition); 57 | } 58 | 59 | public void removeJob(UUID uuid) throws IOException { 60 | jobStore.removeJob(uuid); 61 | } 62 | 63 | public List listJobs() throws IOException { 64 | return jobStore.listAll(); 65 | } 66 | 67 | public JobDefinition getJob(UUID jobUUID) throws IOException { 68 | return jobStore.get(jobUUID); 69 | } 70 | 71 | public JobCompilationResult compile(JobDefinition job, JobDefinitionDesiredstate spec) throws Throwable { 72 | Map inputs = catalogProvider.getInputCatalog(spec.getClusterId()); 73 | AthenaXTableCatalog output = catalogProvider.getOutputCatalog(spec.getClusterId(), job.getOutputs()); 74 | Planner planner = new Planner(inputs, output); 75 | return planner.sql(job.getQuery(), Math.toIntExact(spec.getResource().getVCores())); 76 | } 77 | 78 | @Override 79 | public void onUpdatedInstances(ConcurrentHashMap instances) { 80 | try { 81 | HashMap jobs = JobWatcherUtil.listJobs(jobStore); 82 | HealthCheckReport report = JobWatcherUtil.computeHealthCheckReport(jobs, instances); 83 | ServerContext.INSTANCE.watchdogPolicy().onHealthCheckReport(report); 84 | } catch (IOException e) { 85 | LOG.warn("Failed to run the health check policy ", e); 86 | } 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/JobStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | import com.uber.athenax.backend.api.ExtendedJobDefinition; 22 | import com.uber.athenax.backend.api.JobDefinition; 23 | import com.uber.athenax.backend.server.AthenaXConfiguration; 24 | 25 | import java.io.Closeable; 26 | import java.io.IOException; 27 | import java.util.List; 28 | import java.util.UUID; 29 | 30 | /** 31 | * JobStore is a persistent layer of all the job information. 32 | */ 33 | public interface JobStore extends Closeable { 34 | void open(AthenaXConfiguration conf) throws IOException; 35 | 36 | JobDefinition get(UUID uuid) throws IOException; 37 | 38 | void updateJob(UUID uuid, JobDefinition job) throws IOException; 39 | 40 | void removeJob(UUID uuid) throws IOException; 41 | 42 | List listAll() throws IOException; 43 | } 44 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/WatchdogPolicy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | public interface WatchdogPolicy { 22 | void onHealthCheckReport(HealthCheckReport report); 23 | } 24 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/WatchdogPolicyDefault.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | import com.google.common.collect.Iterables; 22 | import com.uber.athenax.backend.api.ExtendedJobDefinition; 23 | import com.uber.athenax.backend.api.JobDefinition; 24 | import com.uber.athenax.backend.api.JobDefinitionDesiredstate; 25 | import com.uber.athenax.backend.server.ServerContext; 26 | import com.uber.athenax.backend.server.yarn.InstanceInfo; 27 | import com.uber.athenax.backend.server.yarn.InstanceManager; 28 | import com.uber.athenax.vm.compiler.planner.JobCompilationResult; 29 | import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; 30 | import org.apache.hadoop.yarn.exceptions.YarnException; 31 | import org.slf4j.Logger; 32 | import org.slf4j.LoggerFactory; 33 | 34 | import java.io.IOException; 35 | import java.util.List; 36 | import java.util.Map; 37 | 38 | public class WatchdogPolicyDefault implements WatchdogPolicy { 39 | private static final Logger LOG = LoggerFactory.getLogger(WatchdogPolicyDefault.class); 40 | private final InstanceManager instanceManager = ServerContext.INSTANCE.instanceManager(); 41 | private final JobManager jobManager = ServerContext.INSTANCE.jobManager(); 42 | 43 | @Override 44 | public void onHealthCheckReport(HealthCheckReport report) { 45 | for (InstanceInfo info : Iterables.concat(report.spuriousInstances(), report.instancesWithDifferentParameters())) { 46 | try { 47 | instanceManager.killYarnApplication(info.clusterName(), info.appId()); 48 | } catch (ApplicationNotFoundException ignored) { 49 | } catch (IOException | YarnException e) { 50 | LOG.warn("Failed to kill application {}:{}, cause: {}", info.clusterName(), info.appId(), e); 51 | } 52 | } 53 | 54 | for (Map.Entry> e : report.instancesToStart().entrySet()) { 55 | for (JobDefinitionDesiredstate s : e.getValue()) { 56 | final JobCompilationResult res; 57 | JobDefinition job = e.getKey().getDefinition(); 58 | try { 59 | res = jobManager.compile(job, s); 60 | instanceManager.instantiate(s, e.getKey().getUuid(), res); 61 | } catch (Throwable ex) { 62 | LOG.warn("Failed to instantiate the query '{}' on {}", job.getQuery(), s.getClusterId(), ex); 63 | } 64 | } 65 | } 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/jobs/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * The job management component in AthenaX. 21 | */ 22 | package com.uber.athenax.backend.server.jobs; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * The implementation details of various server in AthenaX. 21 | */ 22 | package com.uber.athenax.backend.server; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/ClusterInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import org.apache.hadoop.yarn.client.api.YarnClient; 22 | 23 | class ClusterInfo { 24 | private final String name; 25 | private final YarnClient client; 26 | private final YarnClusterConfiguration conf; 27 | 28 | ClusterInfo(String name, YarnClusterConfiguration conf, YarnClient client) { 29 | this.name = name; 30 | this.client = client; 31 | this.conf = conf; 32 | } 33 | 34 | ClusterInfo(String name, YarnClusterConfiguration conf) { 35 | this.name = name; 36 | this.client = YarnClient.createYarnClient(); 37 | client.init(conf.conf()); 38 | client.start(); 39 | this.conf = conf; 40 | } 41 | 42 | String name() { 43 | return name; 44 | } 45 | 46 | YarnClient client() { 47 | return client; 48 | } 49 | 50 | YarnClusterConfiguration conf() { 51 | return conf; 52 | } 53 | 54 | } 55 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/InstanceInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.uber.athenax.backend.api.InstanceStatus; 22 | import org.apache.hadoop.yarn.api.records.ApplicationId; 23 | 24 | public class InstanceInfo { 25 | private final String clusterName; 26 | private final ApplicationId appId; 27 | private final InstanceMetadata metadata; 28 | private final InstanceStatus status; 29 | 30 | public InstanceInfo(String clusterName, ApplicationId appId, 31 | InstanceMetadata metadata, InstanceStatus status) { 32 | this.metadata = metadata; 33 | this.clusterName = clusterName; 34 | this.appId = appId; 35 | this.status = status; 36 | } 37 | 38 | public String clusterName() { 39 | return clusterName; 40 | } 41 | 42 | public ApplicationId appId() { 43 | return appId; 44 | } 45 | 46 | public InstanceMetadata metadata() { 47 | return metadata; 48 | } 49 | 50 | public InstanceStatus status() { 51 | return status; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/InstanceMetadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.fasterxml.jackson.annotation.JsonProperty; 22 | import com.fasterxml.jackson.core.JsonProcessingException; 23 | import com.fasterxml.jackson.databind.ObjectMapper; 24 | import org.apache.flink.util.Preconditions; 25 | 26 | import java.io.IOException; 27 | import java.util.Objects; 28 | import java.util.UUID; 29 | 30 | /** 31 | * Metadata that is recorded in the application tag of YARN. 32 | */ 33 | public class InstanceMetadata { 34 | static final String YARN_TAG_PREFIX = "athenax_md_"; 35 | private static final ObjectMapper MAPPER = new ObjectMapper(); 36 | 37 | @JsonProperty("i") 38 | private final UUID uuid; 39 | @JsonProperty("d") 40 | private final UUID definition; 41 | 42 | InstanceMetadata() { 43 | this.uuid = null; 44 | this.definition = null; 45 | } 46 | 47 | InstanceMetadata(UUID uuid, UUID definition) { 48 | this.uuid = uuid; 49 | this.definition = definition; 50 | } 51 | 52 | String serialize() { 53 | try { 54 | return YARN_TAG_PREFIX + MAPPER.writeValueAsString(this); 55 | } catch (JsonProcessingException e) { 56 | throw new RuntimeException(e); 57 | } 58 | } 59 | 60 | @Override 61 | public boolean equals(Object obj) { 62 | if (obj instanceof InstanceMetadata) { 63 | InstanceMetadata o = (InstanceMetadata) obj; 64 | return uuid.equals(o.uuid) && definition.equals(o.definition); 65 | } 66 | return false; 67 | } 68 | 69 | @Override 70 | public int hashCode() { 71 | return Objects.hash(uuid, definition); 72 | } 73 | 74 | public UUID uuid() { 75 | return uuid; 76 | } 77 | 78 | public UUID jobDefinition() { 79 | return definition; 80 | } 81 | 82 | static InstanceMetadata deserialize(String s) { 83 | Preconditions.checkArgument(s.startsWith(YARN_TAG_PREFIX)); 84 | String json = s.substring(YARN_TAG_PREFIX.length()); 85 | try { 86 | return MAPPER.readValue(json, InstanceMetadata.class); 87 | } catch (IOException e) { 88 | throw new RuntimeException(e); 89 | } 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/JobConf.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.uber.athenax.backend.api.JobDefinitionResource; 22 | import org.apache.hadoop.fs.Path; 23 | import org.apache.hadoop.yarn.api.records.ApplicationId; 24 | 25 | import java.util.List; 26 | 27 | /** 28 | * JobConf consists of information on how the job should be executed on YARN. 29 | * Such as the resources that need to be localized, the queue, and the amount of resources required to execute the job. 30 | */ 31 | class JobConf { 32 | /** 33 | * The {@link ApplicationId} of the YARN application. 34 | */ 35 | private final ApplicationId yarnAppId; 36 | /** 37 | * The name of the job. 38 | */ 39 | private final String name; 40 | /** 41 | * A list of resources that will be localized for both the JobManager and TaskManager. They will be added 42 | * into the classpaths of both JobManager and the TaskManager as well. 43 | */ 44 | private final List userProvidedJars; 45 | /** 46 | * The name of the YARN queue that executes the job. 47 | */ 48 | private final String queue; 49 | /** 50 | * The number of TaskManager used by the job. 51 | */ 52 | private final long taskManagerCount; 53 | /** 54 | * The number of executor slot per TaskManager. 55 | */ 56 | private final long slotCountPerTaskManager; 57 | /** 58 | * The size of the heap used by each TaskManager. 59 | */ 60 | private final long taskManagerMemoryMb; 61 | 62 | /** 63 | * The Metadata of the instance which will be stored as YARN tags. 64 | */ 65 | private final InstanceMetadata metadata; 66 | 67 | JobConf( 68 | ApplicationId yarnAppId, 69 | String name, 70 | List userProvidedJars, 71 | JobDefinitionResource resource, 72 | InstanceMetadata metadata) { 73 | this.yarnAppId = yarnAppId; 74 | this.name = name; 75 | this.userProvidedJars = userProvidedJars; 76 | this.queue = resource.getQueue(); 77 | this.taskManagerCount = resource.getVCores(); 78 | this.taskManagerMemoryMb = resource.getMemory(); 79 | this.slotCountPerTaskManager = resource.getExecutionSlots(); 80 | this.metadata = metadata; 81 | } 82 | 83 | ApplicationId yarnAppId() { 84 | return yarnAppId; 85 | } 86 | 87 | String name() { 88 | return name; 89 | } 90 | 91 | List userProvidedJars() { 92 | return userProvidedJars; 93 | } 94 | 95 | String queue() { 96 | return queue; 97 | } 98 | 99 | long taskManagerCount() { 100 | return taskManagerCount; 101 | } 102 | 103 | public long slotCountPerTaskManager() { 104 | return slotCountPerTaskManager; 105 | } 106 | 107 | long taskManagerMemoryMb() { 108 | return taskManagerMemoryMb; 109 | } 110 | 111 | InstanceMetadata metadata() { 112 | return metadata; 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/Utils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.uber.athenax.backend.api.InstanceStatus; 22 | import org.apache.flink.annotation.VisibleForTesting; 23 | import org.apache.hadoop.yarn.api.records.ApplicationReport; 24 | import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; 25 | 26 | import java.util.Optional; 27 | import java.util.Set; 28 | 29 | import static com.uber.athenax.backend.server.yarn.InstanceMetadata.YARN_TAG_PREFIX; 30 | 31 | final class Utils { 32 | private Utils() { 33 | } 34 | 35 | /** 36 | * Extract the job definition UUID from the application tag of the YARN application. 37 | */ 38 | @VisibleForTesting 39 | static InstanceMetadata getMetadata(Set tags) { 40 | Optional s = tags.stream().filter(x -> x.startsWith(YARN_TAG_PREFIX)).findFirst(); 41 | if (!s.isPresent()) { 42 | return null; 43 | } 44 | return InstanceMetadata.deserialize(s.get()); 45 | } 46 | 47 | static InstanceInfo extractInstanceInfo(String clusterName, ApplicationReport report) { 48 | InstanceMetadata md = getMetadata(report.getApplicationTags()); 49 | if (md == null) { 50 | return null; 51 | } 52 | 53 | ApplicationResourceUsageReport usage = report.getApplicationResourceUsageReport(); 54 | InstanceStatus stat = new InstanceStatus() 55 | .allocatedVCores((long) usage.getUsedResources().getVirtualCores()) 56 | .allocatedMB((long) usage.getUsedResources().getMemory()) 57 | .clusterId(clusterName) 58 | .applicationId(report.getApplicationId().toString()) 59 | .startedTime(report.getStartTime()) 60 | .runningContainers((long) usage.getNumUsedContainers()) 61 | .trackingUrl(report.getTrackingUrl()) 62 | .state(InstanceStatus.StateEnum.fromValue(report.getYarnApplicationState().toString())); 63 | return new InstanceInfo(clusterName, report.getApplicationId(), md, stat); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/YarnClusterConfiguration.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import org.apache.hadoop.fs.Path; 22 | import org.apache.hadoop.yarn.conf.YarnConfiguration; 23 | 24 | import java.util.Set; 25 | 26 | /** 27 | * {@link YarnClusterConfiguration} consists of information of the YARN cluster. 28 | */ 29 | public class YarnClusterConfiguration { 30 | 31 | /** 32 | * The configuration used by YARN (i.e.,
yarn-site.xml
). 33 | */ 34 | private final YarnConfiguration conf; 35 | 36 | /** 37 | * The home directory of all AthenaX job where all the temporary files for each jobs are stored. 38 | */ 39 | private final String homeDir; 40 | 41 | /** 42 | * The location of the Flink Uber jar. 43 | */ 44 | private final Path flinkUberJar; 45 | 46 | /** 47 | * Additional resources to be localized for both JobManager and TaskManager. 48 | * They will NOT be added into the classpaths. 49 | */ 50 | private final Set resourcesToLocalize; 51 | 52 | /** 53 | * JARs that will be localized and put into the classpaths for bot JobManager and TaskManager. 54 | */ 55 | private final Set systemJars; 56 | 57 | public YarnClusterConfiguration( 58 | YarnConfiguration conf, 59 | String homeDir, 60 | Path flinkUberJar, 61 | Set resourcesToLocalize, 62 | Set systemJars) { 63 | this.conf = conf; 64 | this.homeDir = homeDir; 65 | this.flinkUberJar = flinkUberJar; 66 | this.resourcesToLocalize = resourcesToLocalize; 67 | this.systemJars = systemJars; 68 | } 69 | 70 | YarnConfiguration conf() { 71 | return conf; 72 | } 73 | 74 | public String homeDir() { 75 | return homeDir; 76 | } 77 | 78 | public Path flinkUberJar() { 79 | return flinkUberJar; 80 | } 81 | 82 | public Set resourcesToLocalize() { 83 | return resourcesToLocalize; 84 | } 85 | 86 | public Set systemJars() { 87 | return systemJars; 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /athenax-backend/src/main/java/com/uber/athenax/backend/server/yarn/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This package contains YARN integration of AthenaX. 21 | */ 22 | package com.uber.athenax.backend.server.yarn; 23 | -------------------------------------------------------------------------------- /athenax-backend/src/main/resources/.swagger-codegen-ignore: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/AthenaXConfigurationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server; 20 | 21 | import com.fasterxml.jackson.databind.ObjectMapper; 22 | import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; 23 | import com.uber.athenax.backend.server.jobs.WatchdogPolicyDefault; 24 | 25 | import org.junit.Test; 26 | 27 | import java.io.IOException; 28 | import java.net.URL; 29 | 30 | import static org.junit.Assert.assertEquals; 31 | 32 | public class AthenaXConfigurationTest { 33 | private static final ObjectMapper MAPPER = new ObjectMapper(new YAMLFactory()); 34 | 35 | @Test 36 | public void testParseConfiguration() throws IOException { 37 | URL conf = Thread.currentThread().getContextClassLoader().getResource("athenax-conf-test1.yaml"); 38 | AthenaXConfiguration c = MAPPER.readValue(conf, AthenaXConfiguration.class); 39 | assertEquals("http://localhost:8083", c.masterUri()); 40 | assertEquals(1, c.clusters().size()); 41 | AthenaXConfiguration.YarnCluster foo = c.clusters().get("foo"); 42 | 43 | assertEquals(2, foo.additionalJars().size()); 44 | assertEquals("bar", c.extras().get("foo")); 45 | assertEquals(WatchdogPolicyDefault.class.getCanonicalName(), c.watchdogPolicyImpl()); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/jobs/LevelDBJobStoreTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.jobs; 20 | 21 | import com.uber.athenax.backend.api.JobDefinition; 22 | import com.uber.athenax.backend.server.AthenaXConfiguration; 23 | 24 | import org.junit.Rule; 25 | import org.junit.Test; 26 | import org.junit.rules.TemporaryFolder; 27 | 28 | import java.io.IOException; 29 | import java.util.Collections; 30 | import java.util.UUID; 31 | 32 | import static com.uber.athenax.backend.server.AthenaXExtraConfigOptions.JOBSTORE_LEVELDB_FILE; 33 | import static org.junit.Assert.assertEquals; 34 | import static org.junit.Assert.assertNull; 35 | import static org.junit.Assert.assertTrue; 36 | import static org.mockito.Mockito.doReturn; 37 | import static org.mockito.Mockito.mock; 38 | 39 | public class LevelDBJobStoreTest { 40 | 41 | @Rule 42 | public TemporaryFolder folder = new TemporaryFolder(); 43 | 44 | @Test 45 | public void testLevelDBStore() throws IOException { 46 | UUID jobUUID = UUID.randomUUID(); 47 | JobDefinition def = new JobDefinition() 48 | .query("foo"); 49 | AthenaXConfiguration conf = mock(AthenaXConfiguration.class); 50 | doReturn(Collections.singletonMap(JOBSTORE_LEVELDB_FILE.key(), folder.newFolder("db").getAbsolutePath())) 51 | .when(conf).extras(); 52 | try (LevelDBJobStore db = new LevelDBJobStore()) { 53 | db.open(conf); 54 | assertTrue(db.listAll().isEmpty()); 55 | db.updateJob(jobUUID, def); 56 | assertEquals(def.getQuery(), db.get(jobUUID).getQuery()); 57 | assertEquals(1, db.listAll().size()); 58 | db.removeJob(jobUUID); 59 | assertNull(db.get(jobUUID)); 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/yarn/InstanceManagerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.uber.athenax.backend.api.InstanceState; 22 | import com.uber.athenax.backend.api.InstanceStatus; 23 | import com.uber.athenax.backend.server.AthenaXExtraConfigOptions; 24 | import com.uber.athenax.backend.server.InstanceStateUpdateListener; 25 | import org.apache.hadoop.yarn.api.records.ApplicationId; 26 | import org.apache.hadoop.yarn.client.api.YarnClient; 27 | 28 | import org.junit.Test; 29 | 30 | import java.util.Collections; 31 | import java.util.UUID; 32 | import java.util.concurrent.ScheduledExecutorService; 33 | 34 | import static org.mockito.Matchers.eq; 35 | import static org.mockito.Mockito.mock; 36 | import static org.mockito.Mockito.verify; 37 | 38 | public class InstanceManagerTest { 39 | 40 | @Test 41 | public void testChangeState() throws Exception { 42 | YarnClient client = mock(YarnClient.class); 43 | YarnClusterConfiguration conf = mock(YarnClusterConfiguration.class); 44 | ClusterInfo clusterInfo = new ClusterInfo("foo", conf, client); 45 | UUID app = UUID.randomUUID(); 46 | ApplicationId yarnAppId = mock(ApplicationId.class); 47 | 48 | try (InstanceManager manager = new InstanceManager( 49 | Collections.singletonMap("foo", clusterInfo), 50 | mock(InstanceStateUpdateListener.class), 51 | mock(ScheduledExecutorService.class), 52 | AthenaXExtraConfigOptions.INSTANCE_MANAGER_RESCAN_INTERVAL.defaultValue())) { 53 | InstanceInfo instance = new InstanceInfo("foo", yarnAppId, 54 | mock(InstanceMetadata.class), mock(InstanceStatus.class)); 55 | manager.instances().put(app, instance); 56 | manager.changeState(app, new InstanceState().state(InstanceState.StateEnum.KILLED)); 57 | verify(client).killApplication(eq(yarnAppId)); 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/yarn/InstanceMetadataTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import org.junit.Test; 22 | 23 | import java.util.UUID; 24 | 25 | import static org.junit.Assert.assertEquals; 26 | 27 | public class InstanceMetadataTest { 28 | 29 | @Test 30 | public void testSerialization() { 31 | InstanceMetadata md = new InstanceMetadata(UUID.randomUUID(), UUID.randomUUID()); 32 | InstanceMetadata n = InstanceMetadata.deserialize(md.serialize()); 33 | assertEquals(md, n); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/yarn/JobDeployerITest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import com.uber.athenax.backend.api.JobDefinitionResource; 22 | import com.uber.athenax.backend.server.MiniAthenaXCluster; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.configuration.JobManagerOptions; 25 | import org.apache.hadoop.yarn.api.records.ApplicationId; 26 | import org.apache.hadoop.yarn.api.records.YarnApplicationState; 27 | import org.apache.hadoop.yarn.client.api.YarnClient; 28 | import org.apache.hadoop.yarn.conf.YarnConfiguration; 29 | 30 | import org.junit.Test; 31 | 32 | import java.util.Collections; 33 | import java.util.UUID; 34 | import java.util.concurrent.Executors; 35 | import java.util.concurrent.ScheduledExecutorService; 36 | 37 | import static org.apache.hadoop.yarn.api.records.YarnApplicationState.FINISHED; 38 | import static org.junit.Assert.assertEquals; 39 | 40 | public class JobDeployerITest { 41 | 42 | @Test 43 | public void testCreateAthenaXCluster() throws Exception { 44 | ScheduledExecutorService executor = Executors.newScheduledThreadPool(1); 45 | Configuration flinkConf = new Configuration(); 46 | flinkConf.setString(JobManagerOptions.ADDRESS, "localhost"); 47 | 48 | try (MiniAthenaXCluster cluster = new MiniAthenaXCluster(JobDeployerITest.class.getSimpleName())) { 49 | cluster.start(); 50 | YarnConfiguration conf = cluster.getYarnConfiguration(); 51 | YarnClusterConfiguration clusterConf = cluster.getYarnClusterConf(); 52 | 53 | final ApplicationId appId; 54 | try (YarnClient client = YarnClient.createYarnClient()) { 55 | client.init(conf); 56 | client.start(); 57 | 58 | JobDeployer deployer = new JobDeployer(clusterConf, client, executor, flinkConf); 59 | appId = deployer.createApplication(); 60 | InstanceMetadata md = new InstanceMetadata(UUID.randomUUID(), UUID.randomUUID()); 61 | JobDefinitionResource resource = new JobDefinitionResource() 62 | .queue(null).vCores(1L).executionSlots(1L).memory(2048L); 63 | JobConf jobConf = new JobConf(appId, "test", Collections.emptyList(), resource, md); 64 | deployer.start(JobITestUtil.trivialJobGraph(), jobConf); 65 | 66 | YarnApplicationState state = MiniAthenaXCluster.pollFinishedApplicationState(client, appId); 67 | assertEquals(FINISHED, state); 68 | } 69 | } 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/yarn/JobDeployerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import org.apache.flink.api.common.JobID; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.runtime.instance.ActorGateway; 24 | import org.apache.flink.runtime.jobgraph.JobGraph; 25 | import org.apache.flink.yarn.YarnClusterClient; 26 | import org.apache.hadoop.yarn.client.api.YarnClient; 27 | import org.apache.hadoop.yarn.conf.YarnConfiguration; 28 | import org.junit.Test; 29 | 30 | import scala.concurrent.Future$; 31 | 32 | import java.util.concurrent.ScheduledExecutorService; 33 | 34 | import static org.mockito.Mockito.any; 35 | import static org.mockito.Mockito.doReturn; 36 | import static org.mockito.Mockito.mock; 37 | import static org.mockito.Mockito.verify; 38 | 39 | public class JobDeployerTest { 40 | 41 | @Test 42 | public void testDeployerWithIsolatedConfiguration() throws Exception { 43 | YarnClusterConfiguration clusterConf = mock(YarnClusterConfiguration.class); 44 | doReturn(new YarnConfiguration()).when(clusterConf).conf(); 45 | ScheduledExecutorService executor = mock(ScheduledExecutorService.class); 46 | Configuration flinkConf = new Configuration(); 47 | YarnClient client = mock(YarnClient.class); 48 | JobDeployer deploy = new JobDeployer(clusterConf, client, executor, flinkConf); 49 | AthenaXYarnClusterDescriptor desc = mock(AthenaXYarnClusterDescriptor.class); 50 | 51 | YarnClusterClient clusterClient = mock(YarnClusterClient.class); 52 | doReturn(clusterClient).when(desc).deploy(); 53 | 54 | ActorGateway actorGateway = mock(ActorGateway.class); 55 | doReturn(actorGateway).when(clusterClient).getJobManagerGateway(); 56 | doReturn(Future$.MODULE$.successful(null)).when(actorGateway).ask(any(), any()); 57 | 58 | JobGraph jobGraph = mock(JobGraph.class); 59 | doReturn(JobID.generate()).when(jobGraph).getJobID(); 60 | deploy.start(desc, jobGraph); 61 | 62 | verify(clusterClient).runDetached(jobGraph, null); 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /athenax-backend/src/test/java/com/uber/athenax/backend/server/yarn/JobITestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.backend.server.yarn; 20 | 21 | import org.apache.flink.runtime.jobgraph.JobGraph; 22 | import org.apache.flink.runtime.jobgraph.JobVertex; 23 | import org.apache.flink.runtime.operators.testutils.DummyInvokable; 24 | 25 | final class JobITestUtil { 26 | private JobITestUtil() { 27 | } 28 | 29 | static JobGraph trivialJobGraph() { 30 | JobGraph g = new JobGraph(); 31 | JobVertex v = new JobVertex("1"); 32 | v.setInvokableClass(DummyInvokable.class); 33 | g.addVertex(v); 34 | return g; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /athenax-backend/src/test/resources/athenax-conf-test1.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | athenax.master.uri: http://localhost:8083 20 | extras: 21 | foo: bar 22 | clusters: 23 | foo: 24 | yarn.site.location: http://foo/yarn-site.xml 25 | additional.jars: 26 | - http://foo/foo.jar 27 | - http://foo/bar.jar 28 | - http://foo/foo.jar -------------------------------------------------------------------------------- /athenax-backend/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=INFO,stdout 20 | log4j.threshold=ALL 21 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 22 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 24 | -------------------------------------------------------------------------------- /athenax-tests/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=INFO,stdout 20 | log4j.threshold=ALL 21 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 22 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 24 | -------------------------------------------------------------------------------- /athenax-vm-api/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 15 | 16 | 4.0.0 17 | 18 | com.uber.athenax 19 | athenax 20 | 0.2-SNAPSHOT 21 | ../ 22 | 23 | 24 | athenax-vm-api 25 | AthenaX Public APIs 26 | AthenaX Public APIs 27 | jar 28 | 29 | 30 | 31 | org.apache.flink 32 | flink-core 33 | ${flink.version} 34 | 35 | 36 | org.apache.flink 37 | flink-table_${scala.binary.version} 38 | ${flink.version} 39 | 40 | 41 | org.scala-lang 42 | scala-library 43 | provided 44 | 45 | 46 | 47 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/functions/AthenaXAggregateFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.functions; 20 | 21 | import org.apache.flink.table.functions.AggregateFunction; 22 | import org.apache.flink.table.functions.FunctionContext; 23 | 24 | /** 25 | * Base class for User-Defined Aggregates. 26 | * 27 | *

The behavior of an [[AthenaXAggregateFunction]] can be defined by implementing a series of custom 28 | * methods. An [[AthenaXAggregateFunction]] needs at least three methods: 29 | * - createAccumulator, 30 | * - accumulate, and 31 | * - getValue.

32 | * 33 | *

All these methods muse be declared publicly, not static and named exactly as the names 34 | * mentioned above. The methods createAccumulator and getValue are defined in the 35 | * [[AthenaXAggregateFunction]] functions.

36 | * 37 | *

Example:

38 | * 39 | *
 40 |  *   public class SimpleAverageAccum extends Tuple2<Long,Integer> {
 41 |  *     public long sum = 0;
 42 |  *     public int count = 0;
 43 |  *   }
 44 |  *
 45 |  *   public class SimpleAverage extends AthenaXAggregateFunction<Long, SimpleAverageAccum> {
 46 |  *    public SimpleAverageAccum createAccumulator() {
 47 |  *      return new SimpleAverageAccum();
 48 |  *    }
 49 |  *
 50 |  *    public Long getValue(SimpleAverageAccum accumulator) {
 51 |  *      if (accumulator.count == 0) {
 52 |  *        return null;
 53 |  *      } else {
 54 |  *        return accumulator.sum / accumulator.count;
 55 |  *      }
 56 |  *    }
 57 |  *
 58 |  *    // overloaded accumulate method
 59 |  *    public void accumulate(SimpleAverageAccum accumulator, long iValue) {
 60 |  *      accumulator.sum += iValue;
 61 |  *      accumulator.count += 1;
 62 |  *    }
 63 |  *
 64 |  *    //Overloaded accumulate method
 65 |  *    public void accumulate(SimpleAverageAccum accumulator, int iValue) {
 66 |  *      accumulator.sum += iValue;
 67 |  *      accumulator.count += 1;
 68 |  *    }
 69 |  *  }
 70 |  *
 71 |  * 
72 | * 73 | * @tparam T the type of the aggregation result 74 | * @tparam ACC base class for aggregate Accumulator. The accumulator is used to keep the aggregated 75 | * values which are needed to compute an aggregation result. AggregateFunction 76 | * represents its state using accumulator, thereby the state of the AggregateFunction 77 | * must be put into the accumulator. 78 | */ 79 | public abstract class AthenaXAggregateFunction extends AggregateFunction { 80 | /** 81 | *

Setup method for user-defined function. It can be used for initialization work.

82 | * 83 | *

By default, this method does nothing.

84 | */ 85 | @Override 86 | public void open(FunctionContext context) throws Exception { 87 | super.open(context); 88 | } 89 | 90 | /** 91 | *

Tear-down method for user-defined function. It can be used for clean up work.

92 | * 93 | *

By default, this method does nothing.

94 | */ 95 | @Override 96 | public void close() throws Exception { 97 | super.close(); 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/functions/AthenaXScalarFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.functions; 20 | 21 | import org.apache.flink.table.functions.FunctionContext; 22 | import org.apache.flink.table.functions.ScalarFunction; 23 | 24 | /** 25 | * All user-defined functions (UDFs) in AthenaX needs to inherit from from this class. 26 | * 27 | *

The syntax of the UDFs closely resembles the ones in 28 | * Hive. 29 | *

30 | * 31 | *

The subclasses need to:

32 | *
    33 | *
  • Have a public default constructor.
  • 34 | *
  • Implement at least once public
    eval()
    method that takes 35 | * zero or more parameters.
  • 36 | *
37 | * 38 | */ 39 | public abstract class AthenaXScalarFunction extends ScalarFunction { 40 | 41 | /** 42 | *

Setup method for user-defined function. It can be used for initialization work.

43 | * 44 | *

By default, this method does nothing.

45 | */ 46 | @Override 47 | public void open(FunctionContext context) throws Exception { 48 | super.open(context); 49 | } 50 | 51 | /** 52 | *

Tear-down method for user-defined function. It can be used for clean up work.

53 | * 54 | *

By default, this method does nothing.

55 | */ 56 | @Override 57 | public void close() throws Exception { 58 | super.close(); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/functions/AthenaXTableFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.functions; 20 | 21 | import org.apache.flink.table.functions.FunctionContext; 22 | import org.apache.flink.table.functions.TableFunction; 23 | 24 | /** 25 | * Base class for a user-defined table function (UDTF) in AthenaX. A user-defined table functions works on 26 | * zero, one, or multiple scalar values as input and returns multiple rows as output. 27 | * 28 | *

The behavior of a [[AthenaXTableFunction]] can be defined by implementing a custom evaluation 29 | * method. An evaluation method must be declared publicly, not static and named "eval". 30 | * Evaluation methods can also be overloaded by implementing multiple methods named "eval".

31 | * 32 | *

User-defined functions must have a default constructor and must be instantiable during runtime.

33 | * 34 | *

Example:

35 | * 36 | *
37 |  *   public class Split extends TableFunction<String> {
38 |  *
39 |  *     // implement an "eval" method with as many parameters as you want
40 |  *     public void eval(String str) {
41 |  *       for (String s : str.split(" ")) {
42 |  *         collect(s);   // use collect(...) to emit an output row
43 |  *       }
44 |  *     }
45 |  *
46 |  *     // you can overload the eval method here ...
47 |  *   }
48 |  * 
49 | * 50 | * @tparam T The type of the output row 51 | */ 52 | public abstract class AthenaXTableFunction extends TableFunction { 53 | /** 54 | *

Setup method for user-defined table function. It can be used for initialization work.

55 | * 56 | *

By default, this method does nothing.

57 | */ 58 | @Override 59 | public void open(FunctionContext context) throws Exception { 60 | super.open(context); 61 | } 62 | 63 | /** 64 | *

Tear-down method for user-defined table function. It can be used for clean up work.

65 | * 66 | *

By default, this method does nothing.

67 | */ 68 | @Override 69 | public void close() throws Exception { 70 | super.close(); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/functions/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | /** 19 | * This packages contains the public APIs that can be used to extend AthenaX user-define functions. 20 | */ 21 | package com.uber.athenax.vm.api.functions; 22 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/tables/AthenaXTableCatalog.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.tables; 20 | 21 | import org.apache.flink.table.catalog.ExternalCatalog; 22 | 23 | import java.io.Serializable; 24 | 25 | /** 26 | * A catalog that describes the mappings between SQL tables to data sources. 27 | * 28 | *

AthenaXTableCatalog needs to inherit from {@link Serializable} as the 29 | * actual compilation might happen remotely.

30 | */ 31 | public interface AthenaXTableCatalog extends ExternalCatalog, Serializable { 32 | } 33 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/tables/AthenaXTableCatalogProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.tables; 20 | 21 | import java.util.List; 22 | import java.util.Map; 23 | 24 | /** 25 | * AthenaXTableCatalogProvider provides the catalogs for all clusters. 26 | */ 27 | public interface AthenaXTableCatalogProvider { 28 | /** 29 | * Return the catalogs for input tables for a specific cluster. 30 | * 31 | * @param cluster the name of the cluster 32 | * @return a map from catalog name to the catalog. 33 | */ 34 | Map getInputCatalog(String cluster); 35 | 36 | /** 37 | * Generate a output catalog for a specific cluster. 38 | * 39 | * @param cluster the name of the cluster 40 | * @param outputs customized strings that define the outputs. 41 | * @return a catalog that describes all the outputs. 42 | */ 43 | AthenaXTableCatalog getOutputCatalog(String cluster, List outputs); 44 | } 45 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/tables/AthenaXTableSinkProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.api.tables; 20 | 21 | import org.apache.flink.table.catalog.ExternalCatalogTable; 22 | import org.apache.flink.table.sinks.AppendStreamTableSink; 23 | import org.apache.flink.table.sinks.BatchTableSink; 24 | import org.apache.flink.types.Row; 25 | 26 | import java.io.IOException; 27 | 28 | /** 29 | * A AthenaXTableSinkProvider constructs a data sink from an {@link ExternalCatalogTable} 30 | * so that AthenaX can pipe the data to the external system. 31 | */ 32 | public interface AthenaXTableSinkProvider { 33 | /** 34 | * The scheme that identifies the data sink. 35 | */ 36 | String getType(); 37 | 38 | /** 39 | * Construct an {@link AppendStreamTableSink} for streaming query. 40 | * 41 | * @param table 42 | * The {@link ExternalCatalogTable} that contains all required information to 43 | * construct an {@link AppendStreamTableSink} that accepts the same schema from the table. 44 | */ 45 | AppendStreamTableSink getAppendStreamTableSink(ExternalCatalogTable table) throws IOException; 46 | 47 | /** 48 | * Construct an {@link BatchTableSink} for batch query. 49 | * 50 | * @param table 51 | * The {@link ExternalCatalogTable} that contains all required information to 52 | * construct an {@link BatchTableSink} that accepts the same schema from the table. 53 | */ 54 | BatchTableSink getBatchTableSink(ExternalCatalogTable table) throws IOException; 55 | } 56 | -------------------------------------------------------------------------------- /athenax-vm-api/src/main/java/com/uber/athenax/vm/api/tables/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This packages contains the public APIs that can be used to extend AthenaX. 21 | * including the connectors, table sources/sinks and the catalogs 22 | */ 23 | package com.uber.athenax.vm.api.tables; 24 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/codegen/config.fmpp: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to you under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | data: { 17 | parser: tdd(../data/Parser.tdd) 18 | } 19 | 20 | freemarkerLinks: { 21 | includes: includes/ 22 | } -------------------------------------------------------------------------------- /athenax-vm-compiler/src/codegen/data/Parser.tdd: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to you under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | { 17 | package: "com.uber.athenax.vm.compiler.parser.impl", 18 | class: "SqlParserImpl", 19 | 20 | imports: [ 21 | "com.uber.athenax.vm.compiler.parser.*", 22 | "java.util.*", 23 | ], 24 | 25 | # List of keywords. 26 | keywords: [ 27 | "JAR", 28 | "FILE", 29 | "ARCHIVE", 30 | ], 31 | 32 | nonReservedKeywords: [ 33 | ], 34 | 35 | statementParserMethods: [ 36 | "SqlCreateFunction()" 37 | ], 38 | 39 | # List of methods for parsing custom literals. 40 | literalParserMethods: [ 41 | ], 42 | 43 | # List of methods for parsing custom data types. 44 | dataTypeParserMethods: [ 45 | ], 46 | 47 | # List of methods for parsing extensions to "ALTER " calls. 48 | # Each must accept arguments "(SqlParserPos pos, String scope)". 49 | # Example: "SqlUploadJarNode" 50 | alterStatementParserMethods: [ 51 | ] 52 | 53 | # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls. 54 | # Each must accept arguments "(SqlParserPos pos, boolean replace)". 55 | createStatementParserMethods: [ 56 | ] 57 | 58 | # List of methods for parsing extensions to "DROP" calls. 59 | # Each must accept arguments "(SqlParserPos pos)". 60 | dropStatementParserMethods: [ 61 | ] 62 | 63 | # List of files in @includes directory that have parser method 64 | # implementations for custom SQL statements, literals or types 65 | # given as part of "statementParserMethods", "literalParserMethods" or 66 | # "dataTypeParserMethods". 67 | implementationFiles: [ 68 | "parserImpls.ftl" 69 | ], 70 | 71 | includeCompoundIdentifier: true, 72 | includeBraces: true, 73 | includeAdditionalDeclarations: false 74 | } -------------------------------------------------------------------------------- /athenax-vm-compiler/src/codegen/includes/parserImpls.ftl: -------------------------------------------------------------------------------- 1 | <#-- 2 | // Licensed to the Apache Software Foundation (ASF) under one or more 3 | // contributor license agreements. See the NOTICE file distributed with 4 | // this work for additional information regarding copyright ownership. 5 | // The ASF licenses this file to you under the Apache License, Version 2.0 6 | // (the "License"); you may not use this file except in compliance with 7 | // the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, software 12 | // distributed under the License is distributed on an "AS IS" BASIS, 13 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | // See the License for the specific language governing permissions and 15 | // limitations under the License. 16 | --> 17 | 18 | private void FunctionJarDef(List list) : 19 | { 20 | SqlParserPos pos; 21 | SqlNode uri; 22 | } 23 | { 24 | ( | | ) 25 | { 26 | pos = getPos(); 27 | list.add(StringLiteral()); 28 | } 29 | } 30 | 31 | SqlNodeList FunctionJarDefList() : 32 | { 33 | SqlParserPos pos; 34 | List list = Lists.newArrayList(); 35 | } 36 | { 37 | { pos = getPos(); } 38 | { pos = getPos(); } 39 | FunctionJarDef(list) 40 | ( FunctionJarDef(list) )* 41 | { 42 | return new SqlNodeList(list, pos.plus(getPos())); 43 | } 44 | } 45 | 46 | /** 47 | * CREATE FUNCTION [db_name.]function_name AS class_name 48 | * [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] 49 | */ 50 | SqlCreateFunction SqlCreateFunction() : 51 | { 52 | SqlParserPos pos; 53 | SqlIdentifier dbName = null; 54 | SqlIdentifier funcName; 55 | SqlNode className; 56 | SqlNodeList jarList = null; 57 | } 58 | { 59 | { pos = getPos(); } 60 | 61 | [ 62 | dbName = SimpleIdentifier() 63 | 64 | ] 65 | 66 | funcName = SimpleIdentifier() 67 | 68 | className = StringLiteral() 69 | [ 70 | jarList = FunctionJarDefList() 71 | ] 72 | { 73 | return new SqlCreateFunction(pos, dbName, funcName, className, jarList); 74 | } 75 | } 76 | 77 | 78 | private void SqlStmtList(SqlNodeList list) : 79 | { 80 | } 81 | { 82 | { 83 | list.add(SqlStmt()); 84 | } 85 | } 86 | 87 | SqlNodeList SqlStmtsEof() : 88 | { 89 | SqlParserPos pos; 90 | SqlNodeList stmts; 91 | } 92 | { 93 | { 94 | pos = getPos(); 95 | stmts = new SqlNodeList(pos); 96 | stmts.add(SqlStmt()); 97 | } 98 | ( LOOKAHEAD(2, SqlStmt()) SqlStmtList(stmts) )* 99 | [ ] 100 | { 101 | return stmts; 102 | } 103 | } 104 | 105 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/executor/CompilationResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import org.apache.flink.runtime.jobgraph.JobGraph; 22 | 23 | import java.io.ByteArrayOutputStream; 24 | import java.io.IOException; 25 | import java.io.ObjectOutputStream; 26 | import java.io.Serializable; 27 | 28 | public class CompilationResult implements Serializable { 29 | private JobGraph jobGraph; 30 | private Throwable remoteThrowable; 31 | 32 | public JobGraph jobGraph() { 33 | return jobGraph; 34 | } 35 | 36 | void jobGraph(JobGraph jobGraph) { 37 | this.jobGraph = jobGraph; 38 | } 39 | 40 | void remoteThrowable(Throwable e) { 41 | this.remoteThrowable = e; 42 | } 43 | 44 | public Throwable remoteThrowable() { 45 | return remoteThrowable; 46 | } 47 | 48 | byte[] serialize() { 49 | ByteArrayOutputStream bos = new ByteArrayOutputStream(); 50 | try (ObjectOutputStream os = new ObjectOutputStream(bos)) { 51 | os.writeObject(this); 52 | } catch (IOException e) { 53 | return null; 54 | } 55 | return bos.toByteArray(); 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/executor/ContainedExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import java.io.BufferedOutputStream; 22 | import java.io.File; 23 | import java.io.IOException; 24 | import java.io.ObjectInputStream; 25 | import java.io.OutputStream; 26 | import java.net.InetAddress; 27 | import java.net.InetSocketAddress; 28 | import java.net.ServerSocket; 29 | import java.net.Socket; 30 | import java.util.ArrayList; 31 | import java.util.List; 32 | 33 | public class ContainedExecutor { 34 | 35 | public CompilationResult run(JobDescriptor job) throws IOException { 36 | // HACK: Start a socket to get the results back 37 | // Ideally it is simpler to use stdout / stderr / pipe, 38 | // but many loggers will interfere 39 | try (ServerSocket sock = new ServerSocket()) { 40 | sock.bind(new InetSocketAddress(InetAddress.getLocalHost(), 0)); 41 | ProcessBuilder builder = new ProcessBuilder(buildMainArg(sock.getLocalPort())); 42 | Process proc = builder.start(); 43 | try (OutputStream os = new BufferedOutputStream(proc.getOutputStream())) { 44 | os.write(job.serialize()); 45 | } 46 | 47 | try (Socket client = sock.accept()) { 48 | try (ObjectInputStream is = new ObjectInputStream(client.getInputStream())) { 49 | return (CompilationResult) is.readObject(); 50 | } catch (ClassNotFoundException e) { 51 | throw new IOException(e); 52 | } 53 | } 54 | } 55 | } 56 | 57 | private static List buildMainArg(int port) { 58 | File jvm = new File(new File(System.getProperty("java.home"), "bin"), "java"); 59 | ArrayList ops = new ArrayList<>(); 60 | ops.add(jvm.toString()); 61 | ops.add("-classpath"); 62 | ops.add(System.getProperty("java.class.path")); 63 | String javaLibPath = System.getProperty("java.library.path"); 64 | if (javaLibPath != null) { 65 | ops.add("-Djava.library.path=" + javaLibPath); 66 | } 67 | ops.add(JobCompiler.class.getCanonicalName()); 68 | ops.add(Integer.toString(port)); 69 | return ops; 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/executor/JobDescriptor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalog; 22 | 23 | import java.io.ByteArrayOutputStream; 24 | import java.io.IOException; 25 | import java.io.ObjectOutputStream; 26 | import java.io.Serializable; 27 | import java.util.Map; 28 | 29 | public class JobDescriptor implements Serializable { 30 | private static final long serialVersionUID = -1; 31 | private final Map userDefineFunctions; 32 | private final Map inputs; 33 | private final AthenaXTableCatalog outputs; 34 | private final int parallelism; 35 | 36 | /** 37 | * Stripped down statement that can be recognized by Flink. 38 | */ 39 | private final String sqlStatement; 40 | 41 | public JobDescriptor(Map inputs, 42 | Map userDefineFunctions, 43 | AthenaXTableCatalog outputs, 44 | int parallelism, String sqlStatement) { 45 | this.userDefineFunctions = userDefineFunctions; 46 | this.inputs = inputs; 47 | this.outputs = outputs; 48 | this.parallelism = parallelism; 49 | this.sqlStatement = sqlStatement; 50 | } 51 | 52 | Map udf() { 53 | return userDefineFunctions; 54 | } 55 | 56 | Map inputs() { 57 | return inputs; 58 | } 59 | 60 | AthenaXTableCatalog outputs() { 61 | return outputs; 62 | } 63 | 64 | String sql() { 65 | return sqlStatement; 66 | } 67 | 68 | int parallelism() { 69 | return parallelism; 70 | } 71 | 72 | byte[] serialize() { 73 | ByteArrayOutputStream bos = new ByteArrayOutputStream(); 74 | try (ObjectOutputStream os = new ObjectOutputStream(bos)) { 75 | os.writeObject(this); 76 | } catch (IOException e) { 77 | return null; 78 | } 79 | return bos.toByteArray(); 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/executor/TableSinkProviderRegistry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider; 22 | import org.apache.flink.table.catalog.ExternalCatalogTable; 23 | import org.apache.flink.table.descriptors.ConnectorDescriptorValidator; 24 | import org.apache.flink.table.descriptors.DescriptorProperties; 25 | 26 | import java.util.Collections; 27 | import java.util.HashMap; 28 | import java.util.Map; 29 | import java.util.ServiceLoader; 30 | 31 | @SuppressWarnings("deprecated") 32 | final class TableSinkProviderRegistry { 33 | private static final Map PROVIDERS; 34 | 35 | static { 36 | HashMap providers = new HashMap<>(); 37 | ServiceLoader loaders = 38 | ServiceLoader.load(AthenaXTableSinkProvider.class); 39 | loaders.forEach(x -> providers.put(x.getType(), x)); 40 | PROVIDERS = Collections.unmodifiableMap(providers); 41 | } 42 | 43 | private TableSinkProviderRegistry() { 44 | } 45 | 46 | static AthenaXTableSinkProvider getProvider(ExternalCatalogTable table) { 47 | DescriptorProperties properties = new DescriptorProperties(true); 48 | table.addProperties(properties); 49 | String connectorType = properties.getString(ConnectorDescriptorValidator.CONNECTOR_TYPE()); 50 | return PROVIDERS.get(connectorType); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/executor/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This package implements a restricted environment to compile the SQL. 21 | */ 22 | package com.uber.athenax.vm.compiler.executor; 23 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/parser/SqlCreateFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.parser; 20 | 21 | import org.apache.calcite.sql.SqlCall; 22 | import org.apache.calcite.sql.SqlIdentifier; 23 | import org.apache.calcite.sql.SqlKind; 24 | import org.apache.calcite.sql.SqlNode; 25 | import org.apache.calcite.sql.SqlNodeList; 26 | import org.apache.calcite.sql.SqlOperator; 27 | import org.apache.calcite.sql.SqlSpecialOperator; 28 | import org.apache.calcite.sql.SqlWriter; 29 | import org.apache.calcite.sql.parser.SqlParserPos; 30 | 31 | import java.util.Arrays; 32 | import java.util.List; 33 | 34 | public class SqlCreateFunction extends SqlCall { 35 | private static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("UDF", SqlKind.OTHER_DDL); 36 | private final SqlIdentifier dbName; 37 | private final SqlIdentifier funcName; 38 | private final SqlNode className; 39 | private final SqlNodeList jarList; 40 | 41 | public SqlCreateFunction(SqlParserPos pos, SqlIdentifier dbName, 42 | SqlIdentifier funcName, SqlNode className, SqlNodeList jarList) { 43 | super(pos); 44 | this.dbName = dbName; 45 | this.funcName = funcName; 46 | this.className = className; 47 | this.jarList = jarList; 48 | } 49 | 50 | @Override 51 | public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { 52 | UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec); 53 | u.keyword("CREATE", "FUNCTION"); 54 | if (dbName != null) { 55 | u.node(dbName).keyword("."); 56 | } 57 | u.node(funcName).keyword("AS").node(className); 58 | if (jarList != null) { 59 | u.keyword("USING").nodeList(jarList); 60 | } 61 | } 62 | 63 | @Override 64 | public SqlOperator getOperator() { 65 | return OPERATOR; 66 | } 67 | 68 | @Override 69 | public List getOperandList() { 70 | return Arrays.asList(dbName, funcName, className, jarList); 71 | } 72 | 73 | public SqlIdentifier dbName() { 74 | return dbName; 75 | } 76 | 77 | public SqlIdentifier funcName() { 78 | return funcName; 79 | } 80 | 81 | public SqlNode className() { 82 | return className; 83 | } 84 | 85 | public SqlNodeList jarList() { 86 | return jarList; 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/parser/UnparseUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.parser; 20 | 21 | import org.apache.calcite.sql.SqlNode; 22 | import org.apache.calcite.sql.SqlNodeList; 23 | import org.apache.calcite.sql.SqlWriter; 24 | 25 | import java.util.Arrays; 26 | 27 | class UnparseUtil { 28 | private final SqlWriter writer; 29 | private final int leftPrec; 30 | private final int rightPrec; 31 | 32 | UnparseUtil(SqlWriter writer, int leftPrec, int rightPrec) { 33 | this.writer = writer; 34 | this.leftPrec = leftPrec; 35 | this.rightPrec = rightPrec; 36 | } 37 | 38 | UnparseUtil keyword(String... keywords) { 39 | Arrays.stream(keywords).forEach(writer::keyword); 40 | return this; 41 | } 42 | 43 | UnparseUtil node(SqlNode n) { 44 | n.unparse(writer, leftPrec, rightPrec); 45 | return this; 46 | } 47 | 48 | UnparseUtil nodeList(SqlNodeList l) { 49 | writer.keyword("("); 50 | if (l.size() > 0) { 51 | l.get(0).unparse(writer, leftPrec, rightPrec); 52 | for (int i = 1; i < l.size(); ++i) { 53 | writer.keyword(","); 54 | l.get(i).unparse(writer, leftPrec, rightPrec); 55 | } 56 | } 57 | writer.keyword(")"); 58 | return this; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/parser/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This package consists extensions of the SQL parser. 21 | */ 22 | package com.uber.athenax.vm.compiler.parser; 23 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/planner/JobCompilationResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.planner; 20 | 21 | import org.apache.flink.runtime.jobgraph.JobGraph; 22 | import org.apache.hadoop.fs.Path; 23 | 24 | import java.util.List; 25 | 26 | public class JobCompilationResult { 27 | private final JobGraph jobGraph; 28 | private final List additionalJars; 29 | 30 | JobCompilationResult(JobGraph jobGraph, List additionalJars) { 31 | this.jobGraph = jobGraph; 32 | this.additionalJars = additionalJars; 33 | } 34 | 35 | public JobGraph jobGraph() { 36 | return jobGraph; 37 | } 38 | 39 | public List additionalJars() { 40 | return additionalJars; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/planner/Planner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.planner; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalog; 22 | import com.uber.athenax.vm.compiler.executor.CompilationResult; 23 | import com.uber.athenax.vm.compiler.executor.ContainedExecutor; 24 | import com.uber.athenax.vm.compiler.executor.JobDescriptor; 25 | import com.uber.athenax.vm.compiler.parser.impl.ParseException; 26 | import com.uber.athenax.vm.compiler.parser.impl.SqlParserImpl; 27 | import org.apache.calcite.config.Lex; 28 | import org.apache.calcite.sql.SqlNodeList; 29 | import org.apache.flink.annotation.VisibleForTesting; 30 | import org.apache.hadoop.fs.Path; 31 | 32 | import java.io.StringReader; 33 | import java.util.Map; 34 | import java.util.stream.Collectors; 35 | 36 | public class Planner { 37 | private static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128; 38 | 39 | private final Map inputs; 40 | private final AthenaXTableCatalog outputs; 41 | 42 | public Planner(Map inputs, AthenaXTableCatalog outputs) { 43 | this.inputs = inputs; 44 | this.outputs = outputs; 45 | } 46 | 47 | public JobCompilationResult sql(String sql, int parallelism) throws Throwable { 48 | SqlNodeList stmts = parse(sql); 49 | Validator validator = new Validator(); 50 | validator.validateQuery(stmts); 51 | JobDescriptor job = new JobDescriptor( 52 | inputs, 53 | validator.userDefinedFunctions(), 54 | outputs, 55 | parallelism, 56 | validator.statement().toString()); 57 | // uses contained executor instead of direct compile for: JobCompiler.compileJob(job); 58 | CompilationResult res = new ContainedExecutor().run(job); 59 | 60 | if (res.remoteThrowable() != null) { 61 | throw res.remoteThrowable(); 62 | } 63 | return new JobCompilationResult(res.jobGraph(), 64 | validator.userDefinedFunctions().values().stream().map(Path::new).collect(Collectors.toList())); 65 | } 66 | 67 | @VisibleForTesting 68 | static SqlNodeList parse(String sql) throws ParseException { 69 | // Keep the SQL syntax consistent with Flink 70 | try (StringReader in = new StringReader(sql)) { 71 | SqlParserImpl impl = new SqlParserImpl(in); 72 | 73 | // back tick as the quote 74 | impl.switchTo("BTID"); 75 | impl.setTabSize(1); 76 | impl.setQuotedCasing(Lex.JAVA.quotedCasing); 77 | impl.setUnquotedCasing(Lex.JAVA.unquotedCasing); 78 | impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH); 79 | return impl.SqlStmtsEof(); 80 | } 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/main/java/com/uber/athenax/vm/compiler/planner/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This package compiles the SQL down to a Flink JobGraph. 21 | */ 22 | package com.uber.athenax.vm.compiler.planner; 23 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/test/java/com/uber/athenax/vm/compiler/planner/PlannerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.planner; 20 | 21 | import org.apache.calcite.sql.SqlNodeList; 22 | 23 | import org.junit.Test; 24 | 25 | import static org.junit.Assert.assertEquals; 26 | 27 | public class PlannerTest { 28 | 29 | @Test 30 | public void testCreateFunction() throws Exception { 31 | String sql = "CREATE FUNCTION foo AS 'com.uber.foo';"; 32 | Planner.parse(sql); 33 | 34 | sql = "CREATE FUNCTION foo AS 'com.uber.foo' USING JAR 'mock://foo', JAR 'mock://bar';"; 35 | Planner.parse(sql); 36 | } 37 | 38 | @Test 39 | public void testSetOption() throws Exception { 40 | String sql = "SET flink.enable.checkpoint=1;"; 41 | Planner.parse(sql); 42 | } 43 | 44 | @Test 45 | public void testMultipleStatement() throws Exception { 46 | String sql = "SET flink.enable.checkpoint=1;"; 47 | SqlNodeList list = Planner.parse(sql); 48 | assertEquals(1, list.size()); 49 | 50 | sql = "SET flink.enable.checkpoint=1;\n" 51 | + "SELECT * FROM foo"; 52 | list = Planner.parse(sql); 53 | assertEquals(2, list.size()); 54 | 55 | sql = "SET flink.enable.checkpoint=1;\n" 56 | + "SELECT * FROM foo;"; 57 | list = Planner.parse(sql); 58 | assertEquals(2, list.size()); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /athenax-vm-compiler/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=INFO,stdout 20 | log4j.threshold=ALL 21 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 22 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 24 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 15 | 16 | 4.0.0 17 | 18 | com.uber.athenax 19 | athenax 20 | 0.2-SNAPSHOT 21 | ../../ 22 | 23 | 24 | athenax-vm-connector-common 25 | Common libraries for AthenaX connectors 26 | jar 27 | 28 | 29 | 30 | com.uber.athenax 31 | athenax-vm-api 32 | 33 | 34 | org.apache.flink 35 | flink-streaming-java_${scala.binary.version} 36 | test 37 | 38 | 39 | 40 | 41 | 42 | 43 | org.apache.maven.plugins 44 | maven-jar-plugin 45 | 46 | 47 | 48 | test-jar 49 | 50 | 51 | 52 | default-jar 53 | none 54 | 55 | jar 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/MockAppendStreamTableSink.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.api.java.io.LocalCollectionOutputFormat; 23 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.table.sinks.AppendStreamTableSink; 26 | import org.apache.flink.table.sinks.TableSink; 27 | import org.apache.flink.types.Row; 28 | 29 | import java.util.ArrayList; 30 | import java.util.List; 31 | 32 | class MockAppendStreamTableSink implements AppendStreamTableSink { 33 | private final List rows = new ArrayList<>(); 34 | private final RowTypeInfo type; 35 | 36 | MockAppendStreamTableSink(RowTypeInfo type) { 37 | this.type = type; 38 | } 39 | 40 | @Override 41 | public void emitDataStream(DataStream dataStream) { 42 | dataStream.writeUsingOutputFormat(new LocalCollectionOutputFormat<>(rows)); 43 | } 44 | 45 | @Override 46 | public TypeInformation getOutputType() { 47 | return type; 48 | } 49 | 50 | @Override 51 | public String[] getFieldNames() { 52 | return type.getFieldNames(); 53 | } 54 | 55 | @Override 56 | public TypeInformation[] getFieldTypes() { 57 | return type.getFieldTypes(); 58 | } 59 | 60 | @Override 61 | public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { 62 | RowTypeInfo type = new RowTypeInfo(fieldTypes, fieldNames); 63 | return new MockAppendStreamTableSink(type); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/MockExternalCatalogTable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 22 | import org.apache.flink.table.api.TableSchema; 23 | import org.apache.flink.table.catalog.ExternalCatalogTable; 24 | import org.apache.flink.table.descriptors.ConnectorDescriptor; 25 | import org.apache.flink.table.descriptors.DescriptorProperties; 26 | import org.apache.flink.types.Row; 27 | import scala.Option; 28 | 29 | import java.io.ByteArrayOutputStream; 30 | import java.io.IOException; 31 | import java.io.ObjectOutputStream; 32 | import java.io.Serializable; 33 | import java.util.Base64; 34 | import java.util.List; 35 | 36 | /** 37 | * External Catalog Table based on mock data and mock schema. 38 | */ 39 | public class MockExternalCatalogTable implements Serializable { 40 | static final String TABLE_SCHEMA_CONNECTOR_PROPERTY = "table.schema"; 41 | static final String TABLE_DATA_CONNECTOR_PROPERTY = "table.data"; 42 | static final String CONNECTOR_TYPE = "mock"; 43 | static final int CONNECTOR_VERSION = 1; 44 | 45 | private final RowTypeInfo schema; 46 | private final List data; 47 | 48 | public MockExternalCatalogTable(RowTypeInfo schema, List data) { 49 | this.schema = schema; 50 | this.data = data; 51 | } 52 | 53 | ExternalCatalogTable toExternalCatalogTable() { 54 | TableSchema tableSchema = new TableSchema(schema.getFieldNames(), schema.getFieldTypes()); 55 | ConnectorDescriptor descriptor = new ConnectorDescriptor(CONNECTOR_TYPE, CONNECTOR_VERSION, false) { 56 | @Override 57 | public void addConnectorProperties(DescriptorProperties properties) { 58 | properties.putTableSchema(TABLE_SCHEMA_CONNECTOR_PROPERTY, tableSchema); 59 | properties.putString(TABLE_DATA_CONNECTOR_PROPERTY, serializeRows()); 60 | } 61 | }; 62 | return new ExternalCatalogTable(descriptor, Option.empty(), Option.empty(), Option.empty(), Option.empty()); 63 | } 64 | 65 | private String serializeRows() { 66 | ByteArrayOutputStream bos = new ByteArrayOutputStream(); 67 | try (ObjectOutputStream os = new ObjectOutputStream(bos)) { 68 | os.writeObject(data); 69 | } catch (IOException e) { 70 | return null; 71 | } 72 | return Base64.getEncoder().encodeToString(bos.toByteArray()); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/MockTableSinkProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider; 22 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 23 | import org.apache.flink.table.api.TableSchema; 24 | import org.apache.flink.table.catalog.ExternalCatalogTable; 25 | import org.apache.flink.table.descriptors.DescriptorProperties; 26 | import org.apache.flink.table.sinks.AppendStreamTableSink; 27 | import org.apache.flink.table.sinks.BatchTableSink; 28 | import org.apache.flink.types.Row; 29 | 30 | import java.io.IOException; 31 | 32 | public class MockTableSinkProvider implements AthenaXTableSinkProvider { 33 | @Override 34 | public String getType() { 35 | return "mock"; 36 | } 37 | 38 | @Override 39 | public AppendStreamTableSink getAppendStreamTableSink(ExternalCatalogTable table) throws IOException { 40 | DescriptorProperties params = new DescriptorProperties(true); 41 | table.addProperties(params); 42 | TableSchema tableSchema = params.getTableSchema(MockExternalCatalogTable.TABLE_SCHEMA_CONNECTOR_PROPERTY); 43 | RowTypeInfo type = new RowTypeInfo(tableSchema.getTypes(), tableSchema.getColumnNames()); 44 | return new MockAppendStreamTableSink(type); 45 | } 46 | 47 | @Override 48 | public BatchTableSink getBatchTableSink(ExternalCatalogTable table) throws IOException { 49 | throw new UnsupportedOperationException(); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/MockTableSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 23 | import org.apache.flink.streaming.api.datastream.DataStream; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.table.api.TableSchema; 26 | import org.apache.flink.table.sources.StreamTableSource; 27 | import org.apache.flink.types.Row; 28 | 29 | import java.util.List; 30 | 31 | class MockTableSource implements StreamTableSource { 32 | private final List data; 33 | private final RowTypeInfo type; 34 | private final TableSchema schema; 35 | 36 | MockTableSource(List data, RowTypeInfo type) { 37 | this.data = data; 38 | this.type = type; 39 | this.schema = new TableSchema(type.getFieldNames(), type.getFieldTypes()); 40 | } 41 | 42 | @Override 43 | public DataStream getDataStream(StreamExecutionEnvironment execEnv) { 44 | return execEnv.fromCollection(data); 45 | } 46 | 47 | @Override 48 | public TypeInformation getReturnType() { 49 | return type; 50 | } 51 | 52 | @Override 53 | public TableSchema getTableSchema() { 54 | return schema; 55 | } 56 | 57 | @Override 58 | public String explainSource() { 59 | return "mock"; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/MockTableSourceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 22 | import org.apache.flink.table.api.TableSchema; 23 | import org.apache.flink.table.descriptors.ConnectorDescriptorValidator; 24 | import org.apache.flink.table.descriptors.DescriptorProperties; 25 | import org.apache.flink.table.sources.TableSource; 26 | import org.apache.flink.table.sources.TableSourceFactory; 27 | import org.apache.flink.types.Row; 28 | 29 | import java.io.ByteArrayInputStream; 30 | import java.io.IOException; 31 | import java.io.ObjectInputStream; 32 | import java.util.ArrayList; 33 | import java.util.Base64; 34 | import java.util.HashMap; 35 | import java.util.List; 36 | import java.util.Map; 37 | 38 | import static com.uber.athenax.vm.compiler.executor.MockExternalCatalogTable.CONNECTOR_TYPE; 39 | import static com.uber.athenax.vm.compiler.executor.MockExternalCatalogTable.CONNECTOR_VERSION; 40 | import static com.uber.athenax.vm.compiler.executor.MockExternalCatalogTable.TABLE_DATA_CONNECTOR_PROPERTY; 41 | import static com.uber.athenax.vm.compiler.executor.MockExternalCatalogTable.TABLE_SCHEMA_CONNECTOR_PROPERTY; 42 | 43 | public class MockTableSourceFactory implements TableSourceFactory { 44 | 45 | @Override 46 | public Map requiredContext() { 47 | Map context = new HashMap<>(); 48 | context.put(ConnectorDescriptorValidator.CONNECTOR_TYPE(), CONNECTOR_TYPE); 49 | context.put(ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION(), String.valueOf(CONNECTOR_VERSION)); 50 | return context; 51 | } 52 | 53 | @Override 54 | public List supportedProperties() { 55 | List properties = new ArrayList<>(); 56 | 57 | properties.add(TABLE_DATA_CONNECTOR_PROPERTY); 58 | properties.add(TABLE_SCHEMA_CONNECTOR_PROPERTY + ".#." + "name"); 59 | properties.add(TABLE_SCHEMA_CONNECTOR_PROPERTY + ".#." + "type"); 60 | return properties; 61 | } 62 | 63 | @Override 64 | public TableSource create(Map properties) { 65 | DescriptorProperties params = new DescriptorProperties(true); 66 | params.putProperties(properties); 67 | TableSchema schema = params.getTableSchema(TABLE_SCHEMA_CONNECTOR_PROPERTY); 68 | List rows = deserializeRows(params.getString(TABLE_DATA_CONNECTOR_PROPERTY)); 69 | return new MockTableSource(rows, new RowTypeInfo(schema.getTypes(), schema.getColumnNames())); 70 | } 71 | 72 | private List deserializeRows(String encoded) { 73 | ByteArrayInputStream bis = new ByteArrayInputStream(Base64.getDecoder().decode(encoded)); 74 | try (ObjectInputStream is = new ObjectInputStream(bis)) { 75 | @SuppressWarnings("unchecked") 76 | List res = (List) is.readObject(); 77 | return res; 78 | } catch (ClassNotFoundException | IOException e) { 79 | return null; 80 | } 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/java/com/uber/athenax/vm/compiler/executor/SingleLevelMemoryCatalog.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.compiler.executor; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableCatalog; 22 | import org.apache.flink.table.api.CatalogNotExistException; 23 | import org.apache.flink.table.api.TableNotExistException; 24 | import org.apache.flink.table.catalog.ExternalCatalog; 25 | import org.apache.flink.table.catalog.ExternalCatalogTable; 26 | 27 | import java.io.Serializable; 28 | import java.util.ArrayList; 29 | import java.util.Collections; 30 | import java.util.List; 31 | import java.util.Map; 32 | 33 | public class SingleLevelMemoryCatalog implements AthenaXTableCatalog, Serializable { 34 | private static final long serialVersionUID = -1L; 35 | private final String database; 36 | private final Map tables; 37 | 38 | public SingleLevelMemoryCatalog(String database, Map tables) { 39 | this.database = database; 40 | this.tables = tables; 41 | } 42 | 43 | @Override 44 | public ExternalCatalogTable getTable(String tableName) throws TableNotExistException { 45 | MockExternalCatalogTable table = tables.get(tableName); 46 | if (table == null) { 47 | throw new TableNotExistException(database, "Table " + tableName + " does not exist"); 48 | } 49 | return table.toExternalCatalogTable(); 50 | } 51 | 52 | @Override 53 | public List listTables() { 54 | return new ArrayList<>(tables.keySet()); 55 | } 56 | 57 | @Override 58 | public ExternalCatalog getSubCatalog(String dbName) throws CatalogNotExistException { 59 | throw new CatalogNotExistException(dbName); 60 | } 61 | 62 | @Override 63 | public List listSubCatalogs() { 64 | return Collections.emptyList(); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/resources/META-INF/services/com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | com.uber.athenax.vm.compiler.executor.MockTableSinkProvider -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-common/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | com.uber.athenax.vm.compiler.executor.MockTableSourceFactory 17 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 15 | 16 | 4.0.0 17 | 18 | com.uber.athenax 19 | athenax 20 | 0.2-SNAPSHOT 21 | ../../ 22 | 23 | 24 | athenax-vm-connector-kafka 25 | AthenaX Kafka Connector 26 | jar 27 | 28 | 29 | 30 | com.typesafe 31 | config 32 | 33 | 34 | com.uber.athenax 35 | athenax-vm-api 36 | 37 | 38 | org.mockito 39 | mockito-core 40 | 41 | 42 | org.apache.flink 43 | flink-connector-kafka-0.9_${scala.binary.version} 44 | 45 | 46 | org.apache.flink 47 | flink-json 48 | 49 | 50 | org.apache.flink 51 | flink-metrics-jmx 52 | test 53 | 54 | 55 | org.apache.flink 56 | flink-streaming-java_${scala.binary.version} 57 | 58 | 59 | org.apache.kafka 60 | kafka_${scala.binary.version} 61 | test 62 | 63 | 64 | junit 65 | junit 66 | 67 | 68 | 69 | 70 | 71 | 72 | org.apache.maven.plugins 73 | maven-jar-plugin 74 | 75 | 76 | 77 | test-jar 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/java/com/uber/athenax/vm/connectors/kafka/JsonTableSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSource; 22 | import org.apache.flink.streaming.connectors.kafka.config.StartupMode; 23 | import org.apache.flink.table.api.TableSchema; 24 | import org.apache.flink.table.sources.RowtimeAttributeDescriptor; 25 | 26 | import java.util.Collections; 27 | import java.util.List; 28 | import java.util.Properties; 29 | 30 | class JsonTableSource extends Kafka09JsonTableSource { 31 | static final String KAFKA_JSON_TABLE_SOURCE_TYPE = "kafka+json"; 32 | static final int KAFKA_JSON_TABLE_SOURCE_VERSION = 1; 33 | 34 | JsonTableSource(String topic, Properties properties, TableSchema schema) { 35 | super(topic, properties, schema, schema); 36 | super.setStartupMode(StartupMode.GROUP_OFFSETS); 37 | } 38 | 39 | @Override 40 | public String getProctimeAttribute() { 41 | return "proctime"; 42 | } 43 | 44 | @Override 45 | @SuppressWarnings("unchecked") 46 | public List getRowtimeAttributeDescriptors() { 47 | return (List) Collections.EMPTY_LIST; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/java/com/uber/athenax/vm/connectors/kafka/KafkaConnectorDescriptorValidator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; 22 | 23 | /** 24 | * The configuration keys for the Kafka connectors. 25 | */ 26 | public final class KafkaConnectorDescriptorValidator { 27 | /** 28 | * The prefix of all Kafka configurations that will be passed into 29 | * the Kafka consumer / producer. 30 | */ 31 | public static final String KAFKA_CONFIG_PREFIX = "kafka"; 32 | 33 | /** 34 | * Default partitioner. 35 | */ 36 | public static final String PARTITIONER_CLASS_NAME_DEFAULT = FlinkFixedPartitioner.class.getCanonicalName(); 37 | 38 | /** 39 | * The class name of the Kafka partitioner. The connector will instantiate a new class 40 | * to send the Kafka messages to corresponding partitions. 41 | */ 42 | public static final String PARTITIONER_CLASS_NAME_KEY = "athenax.kafka.partitioner.class"; 43 | 44 | /** 45 | * The name of the Kafka topic to be read or written. 46 | */ 47 | public static final String TOPIC_NAME_KEY = "athenax.kafka.topic.name"; 48 | 49 | /** 50 | * The prefix for Kafka topic schema. 51 | */ 52 | public static final String TOPIC_SCHEMA_KEY = "athenax.kafka.topic.schema"; 53 | 54 | private KafkaConnectorDescriptorValidator() { 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/java/com/uber/athenax/vm/connectors/kafka/KafkaJsonConnector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider; 22 | import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSink; 23 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; 24 | import org.apache.flink.table.catalog.ExternalCatalogTable; 25 | import org.apache.flink.table.descriptors.DescriptorProperties; 26 | import org.apache.flink.table.sinks.AppendStreamTableSink; 27 | import org.apache.flink.table.sinks.BatchTableSink; 28 | import org.apache.flink.types.Row; 29 | 30 | import java.io.IOException; 31 | import java.util.Properties; 32 | 33 | import static com.uber.athenax.vm.connectors.kafka.KafkaConnectorDescriptorValidator.KAFKA_CONFIG_PREFIX; 34 | import static com.uber.athenax.vm.connectors.kafka.KafkaConnectorDescriptorValidator.PARTITIONER_CLASS_NAME_DEFAULT; 35 | import static com.uber.athenax.vm.connectors.kafka.KafkaConnectorDescriptorValidator.PARTITIONER_CLASS_NAME_KEY; 36 | import static com.uber.athenax.vm.connectors.kafka.KafkaConnectorDescriptorValidator.TOPIC_NAME_KEY; 37 | 38 | public class KafkaJsonConnector implements AthenaXTableSinkProvider { 39 | private static final String TYPE = "kafka+json"; 40 | 41 | @Override 42 | public String getType() { 43 | return TYPE; 44 | } 45 | 46 | @Override 47 | public AppendStreamTableSink getAppendStreamTableSink(ExternalCatalogTable table) throws IOException { 48 | DescriptorProperties params = new DescriptorProperties(true); 49 | table.addProperties(params); 50 | String topic = params.getString(TOPIC_NAME_KEY); 51 | Properties conf = new Properties(); 52 | conf.putAll(params.getPrefix(KAFKA_CONFIG_PREFIX)); 53 | String partitionerClass = params.getOptionalString(PARTITIONER_CLASS_NAME_KEY) 54 | .orElse(PARTITIONER_CLASS_NAME_DEFAULT); 55 | FlinkKafkaPartitioner partitioner; 56 | try { 57 | partitioner = KafkaUtils.instantiatePartitioner(partitionerClass); 58 | } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) { 59 | throw new IOException(e); 60 | } 61 | return new Kafka09JsonTableSink(topic, conf, partitioner); 62 | } 63 | 64 | @Override 65 | public BatchTableSink getBatchTableSink(ExternalCatalogTable table) throws IOException { 66 | throw new UnsupportedOperationException(); 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/java/com/uber/athenax/vm/connectors/kafka/KafkaUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 22 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; 23 | import org.apache.flink.table.api.TableSchema; 24 | import org.apache.flink.types.Row; 25 | 26 | import java.util.Map; 27 | import java.util.Properties; 28 | 29 | final class KafkaUtils { 30 | private KafkaUtils() { 31 | } 32 | 33 | static Properties getSubProperties(Map prop, String prefix) { 34 | Properties p = new Properties(); 35 | for (Map.Entry e : prop.entrySet()) { 36 | String key = e.getKey(); 37 | if (key.startsWith(prefix)) { 38 | p.put(key.substring(prefix.length()), e.getValue()); 39 | } 40 | } 41 | return p; 42 | } 43 | 44 | static RowTypeInfo toRowType(TableSchema schema) { 45 | return new RowTypeInfo(schema.getTypes(), schema.getColumnNames()); 46 | } 47 | 48 | static FlinkKafkaPartitioner instantiatePartitioner(String clazzName) 49 | throws ClassNotFoundException, IllegalAccessException, InstantiationException { 50 | @SuppressWarnings("unchecked") 51 | Class> clazz 52 | = (Class>) Class.forName(clazzName); 53 | return clazz.newInstance(); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/java/com/uber/athenax/vm/connectors/kafka/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | /** 20 | * This packages consists of the implementation of the Kafka connectors. 21 | */ 22 | package com.uber.athenax.vm.connectors.kafka; 23 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/resources/META-INF/services/com.uber.athenax.vm.api.tables.AthenaXTableSinkProvider: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | com.uber.athenax.vm.connectors.kafka.KafkaJsonConnector -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | com.uber.athenax.vm.connectors.kafka.JsonTableSourceFactory 17 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/test/java/com/uber/athenax/vm/connectors/kafka/EmbeddedZooKeeper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.apache.commons.io.FileUtils; 22 | import org.apache.zookeeper.server.NIOServerCnxnFactory; 23 | import org.apache.zookeeper.server.ZooKeeperServer; 24 | 25 | import java.io.Closeable; 26 | import java.io.File; 27 | import java.io.IOException; 28 | import java.net.InetSocketAddress; 29 | import java.nio.file.Files; 30 | 31 | public class EmbeddedZooKeeper implements Closeable { 32 | private static final int TICK_TIME = 500; 33 | private final NIOServerCnxnFactory factory; 34 | private final ZooKeeperServer zookeeper; 35 | private final File tmpDir; 36 | private final int port; 37 | 38 | EmbeddedZooKeeper() throws IOException, InterruptedException { 39 | this.tmpDir = Files.createTempDirectory(null).toFile(); 40 | this.factory = new NIOServerCnxnFactory(); 41 | this.zookeeper = new ZooKeeperServer(new File(tmpDir, "data"), new File(tmpDir, "log"), TICK_TIME); 42 | InetSocketAddress addr = new InetSocketAddress("127.0.0.1", 0); 43 | factory.configure(addr, 0); 44 | factory.startup(zookeeper); 45 | this.port = zookeeper.getClientPort(); 46 | } 47 | 48 | public int getPort() { 49 | return port; 50 | } 51 | 52 | @Override 53 | public void close() throws IOException { 54 | zookeeper.shutdown(); 55 | factory.shutdown(); 56 | FileUtils.deleteDirectory(tmpDir); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/test/java/com/uber/athenax/vm/connectors/kafka/FlinkTestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.runtime.jobgraph.JobGraph; 23 | import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; 24 | import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; 25 | import org.apache.flink.streaming.api.graph.StreamGraph; 26 | 27 | final class FlinkTestUtil { 28 | 29 | private FlinkTestUtil() { 30 | } 31 | 32 | static LocalFlinkMiniCluster execute(LocalStreamEnvironment env, 33 | Configuration conf, String jobName) throws Exception { 34 | StreamGraph streamGraph = env.getStreamGraph(); 35 | streamGraph.setJobName(jobName); 36 | JobGraph jobGraph = streamGraph.getJobGraph(); 37 | Configuration configuration = new Configuration(conf); 38 | configuration.addAll(jobGraph.getJobConfiguration()); 39 | configuration.setLong("taskmanager.memory.size", -1L); 40 | configuration.setInteger("taskmanager.numberOfTaskSlots", jobGraph.getMaximumParallelism()); 41 | 42 | LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(configuration, true); 43 | cluster.start(); 44 | cluster.submitJobDetached(jobGraph); 45 | return cluster; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/test/java/com/uber/athenax/vm/connectors/kafka/KafkaTestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import kafka.admin.AdminUtils; 22 | import kafka.common.TopicExistsException; 23 | import kafka.utils.ZkUtils; 24 | import org.apache.flink.util.Preconditions; 25 | import org.apache.kafka.common.security.JaasUtils; 26 | 27 | import java.io.IOException; 28 | import java.net.ServerSocket; 29 | import java.net.URI; 30 | import java.util.Properties; 31 | 32 | public final class KafkaTestUtil { 33 | private static final int ZK_SESSION_TIMEOUT_MS = 10000; 34 | private static final int ZK_CONNECTION_TIMEOUT_MS = 10000; 35 | 36 | private KafkaTestUtil() { 37 | } 38 | 39 | static int getAvailablePort() { 40 | try { 41 | try (ServerSocket socket = new ServerSocket(0)) { 42 | return socket.getLocalPort(); 43 | } 44 | } catch (IOException e) { 45 | throw new RuntimeException("Failed to find available port to use", e); 46 | } 47 | } 48 | 49 | public static boolean createKafkaTopicIfNecessary(String brokerUri, int replFactor, int numPartitions, String topic) 50 | throws IOException { 51 | URI zkUri = URI.create(brokerUri); 52 | Preconditions.checkArgument("zk".equals(zkUri.getScheme())); 53 | String zkServerList = zkUri.getAuthority() + zkUri.getPath(); 54 | 55 | ZkUtils zkUtils = ZkUtils.apply(zkServerList, ZK_SESSION_TIMEOUT_MS, 56 | ZK_CONNECTION_TIMEOUT_MS, JaasUtils.isZkSecurityEnabled()); 57 | try { 58 | if (AdminUtils.topicExists(zkUtils, topic)) { 59 | return false; 60 | } 61 | 62 | try { 63 | AdminUtils.createTopic(zkUtils, topic, numPartitions, replFactor, new Properties()); 64 | } catch (TopicExistsException ignored) { 65 | return false; 66 | } catch (RuntimeException e) { 67 | throw new IOException(e); 68 | } 69 | } finally { 70 | if (zkUtils != null) { 71 | zkUtils.close(); 72 | } 73 | } 74 | return true; 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/test/java/com/uber/athenax/vm/connectors/kafka/KafkaUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.uber.athenax.vm.connectors.kafka; 20 | 21 | import org.junit.Test; 22 | 23 | import java.util.Collections; 24 | import java.util.Map; 25 | import java.util.Properties; 26 | 27 | import static org.junit.Assert.assertEquals; 28 | 29 | public class KafkaUtilsTest { 30 | @Test 31 | public void testGetSubProperties() { 32 | Map parent = Collections.singletonMap("foo.bar", "1"); 33 | Properties children = KafkaUtils.getSubProperties(parent, "foo."); 34 | assertEquals(1, children.size()); 35 | assertEquals("1", children.getProperty("bar")); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /athenax-vm-connectors/athenax-vm-connector-kafka/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=INFO,stdout 20 | log4j.threshold=ALL 21 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 22 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 24 | -------------------------------------------------------------------------------- /dev-support/findbugs-excludes.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | -------------------------------------------------------------------------------- /dev-support/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 28 | 29 | -------------------------------------------------------------------------------- /docs/README.md: -------------------------------------------------------------------------------- 1 | [![ReadTheDocs][doc-img]][doc] 2 | 3 | # AthenaX Documentation 4 | 5 | This directory contains AthenaX documentation hosted by [readthedocs][doc]. 6 | 7 | ## Building 8 | 9 | The documentation is built with [MkDocs](http://www.mkdocs.org/). 10 | You need to have [virtualenv](https://virtualenv.pypa.io/en/stable/) installed. 11 | 12 | Then simply run `make docs`. 13 | 14 | ## Deploying 15 | 16 | Raise a PR on GitHub. Once the PR is approved and merged, 17 | ask one of Athenax maintainers to kick off the build on [readthedocs](https://readthedocs.org/projects/athenax/). 18 | 19 | ## Quick Start 20 | 21 | Please see [here](getting_started.md). 22 | 23 | [doc-img]: https://readthedocs.org/projects/athenax/badge/?version=latest 24 | [doc]: http://athenax.readthedocs.org/en/latest/ 25 | -------------------------------------------------------------------------------- /docs/examples.md: -------------------------------------------------------------------------------- 1 | # Examples 2 | 3 | This section briefly describes a few examples on building streaming analytic applications using SQL. Please read Flink's [Table and SQL API](https://ci.apache.org/projects/flink/flink-docs-release-1.5/dev/table/sql.html) for more details. 4 | 5 | ## Transforming a stream 6 | 7 | The follow query chooses two fields in the table: 8 | 9 | ```sql 10 | SELECT 11 | a, b 12 | FROM 13 | orders 14 | ``` 15 | 16 | ## Aggregation over group windows 17 | 18 | Aggregation over a group window is one of the most common usage pattern in streaming analytics. The following query computes the total number of orders over the last 10 minutes: 19 | 20 | ```sql 21 | SELECT 22 | COUNT(*) 23 | FROM 24 | orders 25 | GROUP BY 26 | HOP(rowtime, INTERVAL '10' MINUTE, INTERVAL '1' MINUTE) 27 | ``` 28 | 29 | ## Using user-defined functions 30 | 31 | AthenaX supports using user-defined functions (UDFs) in the query. The UDF is available in the query once it is registered through the [CREATE FUNCTION](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL) statement. For example, the following query uses the `GetRegion` UDF to compute the region id from the longitude and the latitude fields in the order table: 32 | 33 | ```sql 34 | CREATE FUNCTION 35 | GetRegion 36 | AS 37 | 'com.uber.athenax.geo.GetRegion' 38 | USING JAR 39 | 'http://.../geo.jar'; 40 | 41 | SELECT 42 | GetRegion(lat, lng) 43 | FROM 44 | orders 45 | ``` 46 | -------------------------------------------------------------------------------- /docs/images/athena-logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber-archive/AthenaX/fcafb92d41a6808696abaecad376af6504f45791/docs/images/athena-logo.png -------------------------------------------------------------------------------- /docs/images/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber-archive/AthenaX/fcafb92d41a6808696abaecad376af6504f45791/docs/images/favicon.ico -------------------------------------------------------------------------------- /docs/index.md: -------------------------------------------------------------------------------- 1 | 2 | # AthenaX: SQL-based streaming analytics platform at scale 3 | 4 | Welcome to the documentation portal of AthenaX! 5 | 6 | AthenaX is a streaming analytics platform that enables users to run production-quality, large-scale streaming analytics using Structured Query Language (SQL). AthenaX was released and open sourced by [Uber Technologies][ubeross]. It is capable of scaling across hundreds of machines and processing hundreds of billions of real-time events daily. 7 | 8 | If you can't find what you are looking for, we'd love to hear from you either on [Github](https://github.com/uber/AthenaX/issues), or on our [mailing list](https://groups.google.com/forum/#!forum/athenax-users). 9 | 10 | ## Features 11 | 12 | * Streaming SQL 13 | * Filtering, projecting and combining streams 14 | * Aggregation on [group windows](https://ci.apache.org/projects/flink/flink-docs-release-1.5/dev/table/sql.html#group-windows) over both processing and event time 15 | * User-defined functions (UDF), User-defined aggregation function (UDAF), and User-defined table functions (UDTF) (coming soon) 16 | * Efficient executions through optimizations and code generations 17 | * Mechanisms to automatically fail over across multiple data centers 18 | * Auto scaling for AthenaX jobs 19 | 20 | ## Technical Specs 21 | 22 | * Built on top of [Apache Calcite](http://calcite.apache.org/) and [Apache Flink](http://flink.apache.org/) 23 | * Managed with [YARN](https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) cluster 24 | * [LevelDB](https://github.com/google/leveldb) as persistent storage 25 | 26 | We published a [blog post](https://eng.uber.com/athenax) to describe the design and architecture of AthenaX. 27 | 28 | ## Related links 29 | - [Introducing AthenaX, Uber Engineering’s Open Source Streaming Analytics Platform](https://eng.uber.com/athenax/) 30 | 31 | [ubeross]: http://uber.github.io 32 | -------------------------------------------------------------------------------- /docs/requirements.txt: -------------------------------------------------------------------------------- 1 | mkdocs-material >= 1.5.2 2 | pymdown-extensions >= 2.0 3 | pygments >= 2.2.0 4 | -------------------------------------------------------------------------------- /docs/theme/404.html: -------------------------------------------------------------------------------- 1 | {% extends "base.html" %} 2 | {% block content %} 3 |

404 - Not found

4 | {% endblock %} 5 | -------------------------------------------------------------------------------- /docs/theme/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber-archive/AthenaX/fcafb92d41a6808696abaecad376af6504f45791/docs/theme/__init__.py -------------------------------------------------------------------------------- /docs/theme/assets/images/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber-archive/AthenaX/fcafb92d41a6808696abaecad376af6504f45791/docs/theme/assets/images/favicon.ico -------------------------------------------------------------------------------- /docs/theme/assets/images/favicon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber-archive/AthenaX/fcafb92d41a6808696abaecad376af6504f45791/docs/theme/assets/images/favicon.png -------------------------------------------------------------------------------- /docs/theme/assets/images/icons/bitbucket-670608a71a.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /docs/theme/assets/images/icons/github-1da075986e.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /docs/theme/assets/images/icons/gitlab-5ad3f9f9e5.svg: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /docs/theme/main.html: -------------------------------------------------------------------------------- 1 | {% extends "base.html" %} 2 | -------------------------------------------------------------------------------- /docs/theme/partials/disqus.html: -------------------------------------------------------------------------------- 1 |
2 | 15 | -------------------------------------------------------------------------------- /docs/theme/partials/footer.html: -------------------------------------------------------------------------------- 1 | {% import "partials/language.html" as lang %} 2 | 59 | -------------------------------------------------------------------------------- /docs/theme/partials/header.html: -------------------------------------------------------------------------------- 1 |
2 | 47 |
48 | -------------------------------------------------------------------------------- /docs/theme/partials/language.html: -------------------------------------------------------------------------------- 1 | {% macro t(key) %}{{ { 2 | "edit.link.title": "Edit this page", 3 | "footer.previous": "Previous", 4 | "footer.next": "Next", 5 | "meta.comments": "Comments", 6 | "meta.source": "Source", 7 | "search.placeholder": "Search", 8 | "search.result.placeholder": "Type to start searching", 9 | "search.result.none": "No matching documents", 10 | "search.result.one": "1 matching document", 11 | "search.result.other": "# matching documents", 12 | "source.link.title": "Go to repository", 13 | "toc.title": "Table of contents" 14 | }[key] }}{% endmacro %} 15 | -------------------------------------------------------------------------------- /docs/theme/partials/nav-item.html: -------------------------------------------------------------------------------- 1 | {% set class = "md-nav__item" %} 2 | {% if nav_item.active %} 3 | {% set class = "md-nav__item md-nav__item--active" %} 4 | {% endif %} 5 | {% if nav_item.children %} 6 |
  • 7 | {% if nav_item.active %} 8 | 9 | {% else %} 10 | 11 | {% endif %} 12 | 15 | 28 |
  • 29 | {% elif nav_item == page %} 30 |
  • 31 | {% set toc_ = page.toc %} 32 | 33 | {% if toc_ | first is defined and "\x3ch1 id=" in page.content %} 34 | {% set toc_ = (toc_ | first).children %} 35 | {% endif %} 36 | {% if toc_ | first is defined %} 37 | 40 | {% endif %} 41 | 42 | {{ nav_item.title }} 43 | 44 | {% if toc_ | first is defined %} 45 | {% include "partials/toc.html" %} 46 | {% endif %} 47 |
  • 48 | {% else %} 49 |
  • 50 | 51 | {{ nav_item.title }} 52 | 53 |
  • 54 | {% endif %} 55 | -------------------------------------------------------------------------------- /docs/theme/partials/nav.html: -------------------------------------------------------------------------------- 1 | 25 | -------------------------------------------------------------------------------- /docs/theme/partials/search.html: -------------------------------------------------------------------------------- 1 | {% import "partials/language.html" as lang %} 2 | 22 | -------------------------------------------------------------------------------- /docs/theme/partials/social.html: -------------------------------------------------------------------------------- 1 | {% if config.extra.social %} 2 | 9 | {% endif %} 10 | -------------------------------------------------------------------------------- /docs/theme/partials/source.html: -------------------------------------------------------------------------------- 1 | {% import "partials/language.html" as lang %} 2 | {% set platform = config.extra.repo_icon or config.repo_url %} 3 | {% if "github" in platform %} 4 | {% set repo_type = "github" %} 5 | {% elif "gitlab" in platform %} 6 | {% set repo_type = "gitlab" %} 7 | {% elif "bitbucket" in platform %} 8 | {% set repo_type = "bitbucket" %} 9 | {% else %} 10 | {% set repo_type = "" %} 11 | {% endif %} 12 | {% block repo %} 13 | 14 | {% if repo_type %} 15 |
    16 | 17 | 18 | 19 |
    20 | {% endif %} 21 |
    22 | {{ config.repo_name }} 23 |
    24 |
    25 | {% endblock %} 26 | -------------------------------------------------------------------------------- /docs/theme/partials/tabs-item.html: -------------------------------------------------------------------------------- 1 | {% if nav_item.is_homepage %} 2 |
  • 3 | {% if not page.ancestors | length and nav | selectattr("url", page.url) %} 4 | 5 | {{ nav_item.title }} 6 | 7 | {% else %} 8 | 9 | {{ nav_item.title }} 10 | 11 | {% endif %} 12 |
  • 13 | {% elif nav_item.children and nav_item.children | length > 0 %} 14 | {% set title = title | default(nav_item.title) %} 15 | {% if (nav_item.children | first).children | length > 0 %} 16 | {% set nav_item = nav_item.children | first %} 17 | {% include "partials/tabs-item.html" %} 18 | {% else %} 19 |
  • 20 | {% if nav_item.active %} 21 | 22 | {{ title }} 23 | 24 | {% else %} 25 | 26 | {{ title }} 27 | 28 | {% endif %} 29 |
  • 30 | {% endif %} 31 | {% endif %} 32 | -------------------------------------------------------------------------------- /docs/theme/partials/tabs.html: -------------------------------------------------------------------------------- 1 | {% set class = "md-tabs" %} 2 | {% if page.ancestors | length > 0 %} 3 | {% set class = "md-tabs md-tabs--active" %} 4 | {% endif %} 5 | 14 | -------------------------------------------------------------------------------- /docs/theme/partials/toc-item.html: -------------------------------------------------------------------------------- 1 |
  • 2 | 3 | {{ toc_item.title }} 4 | 5 | {% if toc_item.children %} 6 | 13 | {% endif %} 14 |
  • 15 | -------------------------------------------------------------------------------- /docs/theme/partials/toc.html: -------------------------------------------------------------------------------- 1 | {% import "partials/language.html" as lang %} 2 | 30 | -------------------------------------------------------------------------------- /mkdocs.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | site_name: AthenaX Documentation 17 | edit_uri: edit/master/docs/ 18 | theme_dir: docs/theme 19 | markdown_extensions: 20 | - admonition 21 | - codehilite 22 | - toc(permalink=true) 23 | - pymdownx.betterem(smart_enable=all) 24 | - pymdownx.caret 25 | - pymdownx.critic 26 | - pymdownx.inlinehilite 27 | - pymdownx.mark 28 | - pymdownx.smartsymbols 29 | - pymdownx.superfences 30 | - pymdownx.tasklist(custom_checkbox=true) 31 | - pymdownx.tilde 32 | repo_name: 'uber/AthenaX' 33 | repo_url: 'https://github.com/uber/AthenaX' 34 | google_analytics: 35 | - 'UA-3196917-8' 36 | - 'auto' 37 | pages: 38 | - 'Introduction': 'index.md' 39 | - 'Getting Started' : 'getting_started.md' 40 | - 'Design' : 'design.md' 41 | - 'Examples' : 'examples.md' 42 | --------------------------------------------------------------------------------