├── .gitignore ├── DESIGN.md ├── LICENSE ├── README.md ├── build_and_run.sh ├── pom.xml └── src ├── main ├── conf │ └── app_conf.json ├── java │ └── com │ │ └── datafibers │ │ ├── flinknext │ │ ├── AvroRowDeserializationSchema.java │ │ ├── AvroRowSerializationSchema.java │ │ ├── DFCusterClient.java │ │ ├── DFRemoteStreamEnvironment.java │ │ ├── Kafka010AvroTableSource.java │ │ ├── Kafka09AvroTableSink.java │ │ ├── Kafka09AvroTableSource.java │ │ ├── KafkaAvroTableSink.java │ │ ├── KafkaAvroTableSource.java │ │ ├── KafkaTableSink.java │ │ └── KafkaTableSource.java │ │ ├── model │ │ ├── DFJobPOPJ.java │ │ ├── DFLogPOPJ.java │ │ └── DFModelPOPJ.java │ │ ├── processor │ │ ├── ProcessorConnectKafka.java │ │ ├── ProcessorStreamBack.java │ │ ├── ProcessorTopicSchemaRegistry.java │ │ ├── ProcessorTransformFlink.java │ │ └── ProcessorTransformSpark.java │ │ ├── service │ │ ├── DFDataProcessor.java │ │ ├── DFInitService.java │ │ └── DFWebUI.java │ │ ├── test_tool │ │ ├── AvroConsumerTest.java │ │ ├── AvroConsumerVertx.java │ │ ├── CodeGenFlinkTable.java │ │ ├── FlinkUDF.java │ │ ├── JsonTest.java │ │ ├── KafkaClientTest.java │ │ ├── SimpleAvroTest.java │ │ ├── SparkStructuredStreamingDemo.java │ │ ├── TCFlinkAvroSQL.java │ │ ├── UnitTestSuiteFlink.java │ │ ├── UploadTest.java │ │ └── WordCountStream.java │ │ └── util │ │ ├── CLIParser.java │ │ ├── ConstantApp.java │ │ ├── DFAPIMessage.java │ │ ├── DynamicRunner.java │ │ ├── FlinkAvroSQLClient.java │ │ ├── FlinkAvroTableAPIClient.java │ │ ├── HelpFunc.java │ │ ├── KafkaAdminClient.java │ │ ├── MongoAdminClient.java │ │ ├── Runner.java │ │ └── SchemaRegistryClient.java └── resources │ ├── apidoc │ ├── api_data.js │ ├── api_data.json │ ├── api_project.js │ ├── api_project.json │ ├── css │ │ └── style.css │ ├── img │ │ ├── favicon.ico │ │ ├── glyphicons-halflings-white.png │ │ └── glyphicons-halflings.png │ ├── index.html │ ├── locales │ │ ├── ca.js │ │ ├── de.js │ │ ├── es.js │ │ ├── fr.js │ │ ├── it.js │ │ ├── locale.js │ │ ├── nl.js │ │ ├── pl.js │ │ ├── pt_br.js │ │ ├── ru.js │ │ ├── zh.js │ │ └── zh_cn.js │ ├── main.js │ ├── utils │ │ ├── handlebars_helper.js │ │ └── send_sample_request.js │ └── vendor │ │ ├── bootstrap-responsive.min.css │ │ ├── bootstrap.min.css │ │ ├── bootstrap.min.js │ │ ├── diff_match_patch.min.js │ │ ├── handlebars.min.js │ │ ├── jquery.min.js │ │ ├── lodash.min.js │ │ ├── path-to-regexp │ │ ├── LICENSE │ │ └── index.js │ │ ├── polyfill.js │ │ ├── prettify.css │ │ ├── prettify │ │ ├── lang-apollo.js │ │ ├── lang-basic.js │ │ ├── lang-clj.js │ │ ├── lang-css.js │ │ ├── lang-dart.js │ │ ├── lang-erlang.js │ │ ├── lang-go.js │ │ ├── lang-hs.js │ │ ├── lang-lisp.js │ │ ├── lang-llvm.js │ │ ├── lang-lua.js │ │ ├── lang-matlab.js │ │ ├── lang-ml.js │ │ ├── lang-mumps.js │ │ ├── lang-n.js │ │ ├── lang-pascal.js │ │ ├── lang-proto.js │ │ ├── lang-r.js │ │ ├── lang-rd.js │ │ ├── lang-scala.js │ │ ├── lang-sql.js │ │ ├── lang-tcl.js │ │ ├── lang-tex.js │ │ ├── lang-vb.js │ │ ├── lang-vhdl.js │ │ ├── lang-wiki.js │ │ ├── lang-xq.js │ │ ├── lang-yaml.js │ │ ├── prettify.css │ │ ├── prettify.js │ │ └── run_prettify.js │ │ ├── require.min.js │ │ ├── semver.min.js │ │ └── webfontloader.js │ ├── dfa │ ├── asset-manifest.json │ ├── favicon.ico │ ├── index.html │ ├── service-worker.js │ └── static │ │ ├── css │ │ ├── main.5631a031.css │ │ └── main.5631a031.css.map │ │ └── js │ │ ├── main.a334a047.js │ │ └── main.a334a047.js.map │ ├── import │ └── df_installed.json │ ├── landing │ ├── README.txt │ ├── assets │ │ ├── css │ │ │ ├── font-awesome.min.css │ │ │ ├── ie8.css │ │ │ ├── ie9.css │ │ │ └── main.css │ │ ├── fonts │ │ │ ├── FontAwesome.otf │ │ │ ├── fontawesome-webfont.eot │ │ │ ├── fontawesome-webfont.svg │ │ │ ├── fontawesome-webfont.ttf │ │ │ ├── fontawesome-webfont.woff │ │ │ └── fontawesome-webfont.woff2 │ │ ├── js │ │ │ ├── ie │ │ │ │ ├── html5shiv.js │ │ │ │ └── respond.min.js │ │ │ └── main.js │ │ └── sass │ │ │ ├── base │ │ │ ├── _bg.scss │ │ │ ├── _page.scss │ │ │ └── _typography.scss │ │ │ ├── components │ │ │ ├── _button.scss │ │ │ ├── _form.scss │ │ │ ├── _icon.scss │ │ │ ├── _list.scss │ │ │ └── _section.scss │ │ │ ├── ie8.scss │ │ │ ├── ie9.scss │ │ │ ├── layout │ │ │ ├── _footer.scss │ │ │ ├── _header.scss │ │ │ └── _signup-form.scss │ │ │ ├── libs │ │ │ ├── _functions.scss │ │ │ ├── _mixins.scss │ │ │ ├── _skel.scss │ │ │ └── _vars.scss │ │ │ └── main.scss │ ├── images │ │ ├── bg02.jpg │ │ ├── bg03.jpg │ │ ├── bg04.jpg │ │ ├── bg05.jpg │ │ └── bg06.jpg │ └── index.html │ └── log4j.properties └── test ├── java └── com │ └── datafibers │ └── service │ ├── MyFirstVerticleTest.java │ └── MyRestIT.java └── resources └── my-it-config.json /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | 3 | # Mobile Tools for Java (J2ME) 4 | .mtj.tmp/ 5 | 6 | # Package Files # 7 | *.jar 8 | *.war 9 | *.ear 10 | 11 | /target/ 12 | 13 | .idea 14 | .settings 15 | .project 16 | .classpath 17 | .svn 18 | .iml 19 | dependency-reduced-pom.xml 20 | .DS_Store 21 | .vertx/ 22 | /target/ 23 | -------------------------------------------------------------------------------- /DESIGN.md: -------------------------------------------------------------------------------- 1 | # DataFibers Data Services Design 2 | 3 | ## Connect 4 | 5 | 6 | 7 | ## Transform 8 | All final data will kept as Avro format. All interim data will keep as Json format since we'll have less efforts for schema 9 | * Avro_Avro_SQL: Simple data transformation from Avro to Avro format 10 | * Avro_Json_SQL: Usually play as preparing transformation for other Json transformations 11 | * Json_Avro_SQL: Convert Json data to Avro data with schema created 12 | * Json_Json_Router: Data to different topics on different conditions 13 | * Json_Json_Fiter: Filter data for specific conditions 14 | * Json_Json_LKP 15 | * Json_Json_Exp 16 | * Json_Json_Agg 17 | * Json_Json_Copy 18 | * Json_Json_Union 19 | * Json_Json_Concat 20 | * Json_Json_Merge 21 | * Json_Json_Tag: Add/Tag additional data to the original topic 22 | 23 | 24 | 25 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # DataFibers Data Services 2 | DataFibers (DF) - A pure streaming processing application on Kafka and Flink. 3 | The DF processor has two components defined to deal with stream ETL (Extract, Transform, and Load). 4 | * **Connects** is to leverage Kafka Connect REST API on Confluent to landing or publishing data in or out of Apache Kafka. 5 | * **Transforms** is to leverage streaming processing engine, such as Apache Flink, for data transformation. 6 | 7 | ## Building 8 | 9 | You build the project using: 10 | 11 | ``` 12 | mvn clean package 13 | ``` 14 | 15 | ## Testing 16 | 17 | The application is tested using [vertx-unit](http://vertx.io/docs/vertx-unit/java/). 18 | 19 | ## Packaging 20 | 21 | The application is packaged as a _fat jar_, using the 22 | [Maven Shade Plugin](https://maven.apache.org/plugins/maven-shade-plugin/). 23 | 24 | ## Running 25 | 26 | Once packaged, just launch the _fat jar_ as follows ways 27 | 28 | * Default with no parameters to launch standalone mode with web ui. 29 | ``` 30 | java -jar df-data-service--SNAPSHOT-fat.jar 31 | ``` 32 | 33 | * For more running features checking help option 34 | ``` 35 | java -jar df-data-service--SNAPSHOT-fat.jar -h 36 | ``` 37 | 38 | ## Web UI 39 | http://localhost:8000/ or http://localhost:8000/dfa/ 40 | 41 | 42 | ## Manual 43 | https://datafibers-community.gitbooks.io/datafibers-complete-guide/content/ 44 | 45 | ## Demo 46 | [![DataFibers Demo](http://img.youtube.com/vi/S1CsAf5qCBQ/0.jpg)](http://www.youtube.com/watch?v=S1CsAf5qCBQ "DataFibers Demo") 47 | 48 | ## Todo 49 | - [x] Fetch all installed connectors/plugins in regularly frequency 50 | - [x] Need to report connector or job status 51 | - [x] Need an initial method to import all available|paused|running connectors from kafka connect 52 | - [x] Add Flink Table API engine 53 | - [ ] Add memory LKP 54 | - [x] Add Connects, Transforms Logging URL 55 | - [ ] Add to generic function to do connector validation before creation 56 | - [x] Add submit other job actions, such as start, hold, etc 57 | - [ ] Add Spark Structure Streaming 58 | - [X] Topic visualization 59 | - [ ] Launch 3rd party jar 60 | - [ ] Job level control, schedule, and metrics 61 | -------------------------------------------------------------------------------- /build_and_run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | mvn package -DskipTests; java -jar target/df-data-service-*.jar 3 | -------------------------------------------------------------------------------- /src/main/conf/app_conf.json: -------------------------------------------------------------------------------- 1 | { 2 | "debug.mode":"false", 3 | "rest.port.df.processor":8080, 4 | "http.port.df.processor":8000, 5 | 6 | "db.name":"DEFAULT_DB", 7 | "db.collection.name":"df_processor", 8 | "db.collection_installed.name":"df_installed", 9 | "db.metadata.collection.name":"df_meta", 10 | "db.log.collection.name":"df_log", 11 | "db.log.cleanup.on.start":"true", 12 | "repo.connection.string":"mongodb://localhost:27017", 13 | 14 | "kafka.connect.enable":"true", 15 | "kafka.connect.rest.host":"localhost", 16 | "kafka.server.port":9092, 17 | "kafka.schema.registry.rest.port":8081, 18 | "kafka.connect.rest.port":8083, 19 | 20 | "kafka.connect.import.start":"true", 21 | "kafka.topic.df.metadata":"df_meta", 22 | 23 | "transform.engine.flink.enable":"true", 24 | "flink.server.host":"localhost", 25 | "flink.rest.server.port":8001, 26 | 27 | "transform.engine.spark.enable":"true", 28 | "spark.livy.server.host":"localhost", 29 | "spark.livy.server.port":8998, 30 | 31 | "webhdfs.server.host":"localhost", 32 | "webhdfs.server.port":8998, 33 | 34 | "df.metadata.schema":"{\"type\":\"record\",\"name\": \"df_meta\",\"fields\":[{\"name\": \"cuid\", \"type\": \"string\"},{\"name\": \"file_name\", \"type\": \"string\"},{\"name\": \"file_size\", \"type\": \"string\"}, {\"name\": \"file_owner\", \"type\": \"string\"},{\"name\": \"last_modified_timestamp\", \"type\": \"string\"},{\"name\": \"current_timestamp\", \"type\": \"string\"},{\"name\": \"current_timemillis\", \"type\": \"long\"},{\"name\": \"stream_offset\", \"type\": \"string\"},{\"name\": \"topic_sent\", \"type\": \"string\"},{\"name\": \"schema_subject\", \"type\": \"string\"},{\"name\": \"schema_version\", \"type\": \"string\"},{\"name\": \"status\", \"type\": \"string\"}]}" 35 | } -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/AvroRowSerializationSchema.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.flinknext; 2 | 3 | import com.datafibers.util.ConstantApp; 4 | import org.apache.avro.Schema; 5 | import org.apache.avro.generic.GenericData; 6 | import org.apache.avro.generic.GenericDatumWriter; 7 | import org.apache.avro.generic.GenericRecord; 8 | import org.apache.avro.io.*; 9 | import org.apache.avro.util.Utf8; 10 | import org.apache.flink.api.java.tuple.Tuple2; 11 | import org.apache.flink.streaming.util.serialization.SerializationSchema; 12 | import org.apache.flink.types.Row; 13 | import org.apache.log4j.Logger; 14 | import java.io.ByteArrayOutputStream; 15 | import java.io.IOException; 16 | import java.nio.ByteBuffer; 17 | import java.util.List; 18 | import java.util.Properties; 19 | 20 | 21 | /** 22 | * Deserialization schema from AVRO to {@link Row}. 23 | * 24 | *

Deserializes the byte[] messages as a AVROject and reads 25 | * the specified fields. 26 | * 27 | *

Failure during deserialization are forwarded as wrapped IOExceptions. 28 | */ 29 | public class AvroRowSerializationSchema implements SerializationSchema> { 30 | 31 | private static final long serialVersionUID = 4330538776656642780L; 32 | private static final Logger LOG = Logger.getLogger(AvroRowSerializationSchema.class); 33 | 34 | 35 | /** 36 | * Low-level class for serialization of Avro values. 37 | */ 38 | 39 | protected final Properties properties; 40 | 41 | 42 | /** Generic Avro Schema reader for the row */ 43 | private transient DatumWriter writer; 44 | 45 | /** TODO - When schema changes, the Source table does not need to be recreated.*/ 46 | 47 | /** 48 | * Creates a AVRO serializtion schema for the given schema. 49 | * 50 | * @param schema Names of AVRO fields to parse. 51 | */ 52 | 53 | 54 | public AvroRowSerializationSchema(Properties properties) { 55 | this.properties = properties; 56 | } 57 | @Override 58 | public byte[] serialize(Tuple2 row) { 59 | try { 60 | int schemaId = Integer.parseInt(properties.get(ConstantApp.PK_SCHEMA_ID_OUTPUT).toString()); 61 | 62 | ByteArrayOutputStream out = new ByteArrayOutputStream(); 63 | out.write(ConstantApp.MAGIC_BYTE); 64 | out.write(ByteBuffer.allocate(ConstantApp.idSize).putInt(schemaId).array()); 65 | 66 | BinaryEncoder encoder = EncoderFactory.get().directBinaryEncoder(out, null); 67 | Schema schema = new Schema.Parser().parse(properties.get(ConstantApp.PK_SCHEMA_STR_OUTPUT).toString()); 68 | 69 | DatumWriter writer = new GenericDatumWriter(schema); 70 | writer.write(convertToRecord(schema, row.f1), encoder); //TODO TO CHECK 71 | encoder.flush(); 72 | 73 | byte[] bytes = out.toByteArray(); 74 | out.close(); 75 | return bytes; 76 | 77 | } catch (IOException t) { 78 | t.printStackTrace(); 79 | 80 | throw new RuntimeException("Failed to serialize Row.", t); 81 | } 82 | 83 | 84 | } 85 | 86 | /** 87 | * Converts a (nested) Flink Row into Avro's {@link GenericRecord}. 88 | * Strings are converted into Avro's {@link Utf8} fields. 89 | */ 90 | private static Object convertToRecord(Schema schema, Object rowObj) { 91 | 92 | if (rowObj instanceof Row) { 93 | 94 | // records can be wrapped in a union 95 | if (schema.getType() == Schema.Type.UNION) { 96 | final List types = schema.getTypes(); 97 | if (types.size() == 2 && types.get(0).getType() == Schema.Type.NULL && types.get(1).getType() == Schema.Type.RECORD) { 98 | schema = types.get(1); 99 | } 100 | else if (types.size() == 2 && types.get(0).getType() == Schema.Type.RECORD && types.get(1).getType() == Schema.Type.NULL) { 101 | schema = types.get(0); 102 | } 103 | else { 104 | throw new RuntimeException("Currently we only support schemas of the following form: UNION[null, RECORD] or UNION[RECORD, NULL] Given: " + schema); 105 | } 106 | } else if (schema.getType() != Schema.Type.RECORD) { 107 | throw new RuntimeException("Record type for row type expected. But is: " + schema); 108 | } 109 | final List fields = schema.getFields(); 110 | final GenericRecord record = new GenericData.Record(schema); 111 | final Row row = (Row) rowObj; 112 | for (int i = 0; i < fields.size(); i++) { 113 | final Schema.Field field = fields.get(i); 114 | record.put(field.pos(), convertToRecord(field.schema(), row.getField(i))); 115 | } 116 | return record; 117 | } else if (rowObj instanceof String) { 118 | return new Utf8((String) rowObj); 119 | } else { 120 | return rowObj; 121 | } 122 | } 123 | 124 | } 125 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/DFCusterClient.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.flinknext; 2 | 3 | import java.net.URISyntaxException; 4 | import java.net.URL; 5 | import java.util.Collections; 6 | import java.util.List; 7 | 8 | import org.apache.flink.api.common.JobSubmissionResult; 9 | import org.apache.flink.client.program.ClusterClient; 10 | import org.apache.flink.client.program.ProgramInvocationException; 11 | import org.apache.flink.configuration.ConfigConstants; 12 | import org.apache.flink.configuration.Configuration; 13 | import org.apache.flink.core.fs.Path; 14 | import org.apache.flink.optimizer.plan.FlinkPlan; 15 | import org.apache.flink.optimizer.plan.OptimizedPlan; 16 | import org.apache.flink.optimizer.plan.StreamingPlan; 17 | import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; 18 | import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus; 19 | import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; 20 | import org.apache.flink.runtime.instance.ActorGateway; 21 | import org.apache.flink.runtime.jobgraph.JobGraph; 22 | import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; 23 | 24 | import scala.concurrent.Await; 25 | import scala.concurrent.Future; 26 | 27 | import com.datafibers.model.DFJobPOPJ; 28 | 29 | /** 30 | * For Unit Test Only 31 | * This is customized DF cluster client for TESTING communication with an Flink standalone (on-premise) cluster or an existing cluster that has been 32 | * brought up independently of a specific job. The runWithDFObj is added to pass DFPOPJ into job execution. The jobId is set to the jobConfig 33 | * immediately before submit Flink job. 34 | */ 35 | public class DFCusterClient extends ClusterClient { 36 | 37 | public DFCusterClient(Configuration config) throws Exception { 38 | super(config); 39 | } 40 | 41 | @Override 42 | public void waitForClusterToBeReady() {} 43 | 44 | @SuppressWarnings( "deprecation" ) 45 | @Override 46 | public String getWebInterfaceURL() { 47 | String host = this.getJobManagerAddress().getHostString(); 48 | int port = getFlinkConfiguration().getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 49 | ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT); 50 | return "http://" + host + ":" + port; 51 | } 52 | 53 | @Override 54 | public GetClusterStatusResponse getClusterStatus() { 55 | ActorGateway jmGateway; 56 | try { 57 | jmGateway = getJobManagerGateway(); 58 | Future future = jmGateway.ask(GetClusterStatus.getInstance(), timeout); 59 | Object result = Await.result(future, timeout); 60 | if (result instanceof GetClusterStatusResponse) { 61 | return (GetClusterStatusResponse) result; 62 | } else { 63 | throw new RuntimeException("Received the wrong reply " + result + " from cluster."); 64 | } 65 | } catch (Exception e) { 66 | throw new RuntimeException("Couldn't retrieve the Cluster status.", e); 67 | } 68 | } 69 | 70 | @Override 71 | public List getNewMessages() { 72 | return Collections.emptyList(); 73 | } 74 | 75 | @Override 76 | public String getClusterIdentifier() { 77 | // Avoid blocking here by getting the address from the config without resolving the address 78 | return "DF Standalone cluster with JobManager at " + this.getJobManagerAddress(); 79 | } 80 | 81 | @Override 82 | public int getMaxSlots() { 83 | return -1; 84 | } 85 | 86 | @Override 87 | protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) 88 | throws ProgramInvocationException { 89 | if (isDetached()) { 90 | return super.runDetached(jobGraph, classLoader); 91 | } else { 92 | return super.run(jobGraph, classLoader); 93 | } 94 | } 95 | 96 | @Override 97 | protected void finalizeCluster() {} 98 | 99 | 100 | public JobSubmissionResult runWithDFObj( 101 | FlinkPlan compiledPlan, List libraries, List classpaths, ClassLoader classLoader, DFJobPOPJ dfJobPOPJ) throws ProgramInvocationException { 102 | return runWithDFObj(compiledPlan, libraries, classpaths, classLoader, SavepointRestoreSettings.none(), dfJobPOPJ); 103 | } 104 | 105 | public JobSubmissionResult runWithDFObj(FlinkPlan compiledPlan, 106 | List libraries, List classpaths, ClassLoader classLoader, SavepointRestoreSettings savepointSettings, DFJobPOPJ dfJobPOPJ) 107 | throws ProgramInvocationException { 108 | JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointSettings); 109 | // Keep the jobID to DFPOPJ 110 | dfJobPOPJ.setFlinkIDToJobConfig(job.getJobID().toString()); 111 | return submitJob(job, classLoader); 112 | } 113 | 114 | 115 | private JobGraph getJobGraph(FlinkPlan optPlan, List jarFiles, List classpaths, SavepointRestoreSettings savepointSettings) { 116 | JobGraph job; 117 | if (optPlan instanceof StreamingPlan) { 118 | job = ((StreamingPlan) optPlan).getJobGraph(); 119 | job.setSavepointRestoreSettings(savepointSettings); 120 | } else { 121 | JobGraphGenerator gen = new JobGraphGenerator(this.flinkConfig); 122 | job = gen.compileJobGraph((OptimizedPlan) optPlan); 123 | } 124 | 125 | for (URL jar : jarFiles) { 126 | try { 127 | job.addJar(new Path(jar.toURI())); 128 | } catch (URISyntaxException e) { 129 | throw new RuntimeException("URL is invalid. This should not happen.", e); 130 | } 131 | } 132 | 133 | job.setClasspaths(classpaths); 134 | 135 | return job; 136 | } 137 | 138 | 139 | @Override 140 | public boolean hasUserJarsInClassPath(List arg0) { 141 | // TODO Auto-generated method stub 142 | return false; 143 | } 144 | 145 | } 146 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/Kafka010AvroTableSource.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.flinknext; 2 | 3 | import org.apache.flink.api.common.typeinfo.TypeInformation; 4 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; 5 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; 6 | import org.apache.flink.streaming.util.serialization.DeserializationSchema; 7 | import org.apache.flink.table.sources.StreamTableSource; 8 | import org.apache.flink.types.Row; 9 | 10 | import java.util.Properties; 11 | 12 | /** 13 | * Kafka {@link StreamTableSource} for Kafka 0.10. 14 | */ 15 | public class Kafka010AvroTableSource extends KafkaAvroTableSource { 16 | 17 | /** 18 | * Creates a Kafka 0.10 AVRO {@link StreamTableSource}. 19 | * 20 | * @param topic Kafka topic to consume. 21 | * @param properties Properties for the Kafka consumer. 22 | * @param fieldNames Row field names. 23 | * @param fieldTypes Row field types. 24 | */ 25 | public Kafka010AvroTableSource( 26 | String topic, 27 | Properties properties, 28 | String[] fieldNames, 29 | TypeInformation[] fieldTypes) { 30 | 31 | super(topic, properties, fieldNames, fieldTypes); 32 | } 33 | 34 | /** 35 | * Creates a Kafka 0.10 AVRO fields name and type are derived from schema info in properties 36 | * @param topic - topic in Kafka to map the table source 37 | * @param properties - list of properties to connect Kafka, etc 38 | * 39 | *

Following property need to set ahead of using. 40 | *

properties.setProperty("schema.subject", "test-value"); // Subject Name for the SchemaRegistry 41 | *

properties.setProperty("schema.registry", "localhost:8081"); // Host and port for the SchemaRegistry 42 | *

properties.setProperty("static.avro.schema", STATIC_USER_SCHEMA); // Schema string when Schema is static. With Static Schema, SchemaRegistry does not have to be used. 43 | */ 44 | public Kafka010AvroTableSource(String topic, Properties properties) { 45 | 46 | super(topic, properties); 47 | } 48 | 49 | /** 50 | * Creates a Kafka 0.10 AVRO {@link StreamTableSource}. 51 | * 52 | * @param topic Kafka topic to consume. 53 | * @param properties Properties for the Kafka consumer. 54 | * @param fieldNames Row field names. 55 | * @param fieldTypes Row field types. 56 | */ 57 | public Kafka010AvroTableSource( 58 | String topic, 59 | Properties properties, 60 | String[] fieldNames, 61 | Class[] fieldTypes) { 62 | 63 | super(topic, properties, fieldNames, fieldTypes); 64 | } 65 | 66 | @Override 67 | FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { 68 | return new FlinkKafkaConsumer010<>(topic, deserializationSchema, properties); 69 | } 70 | 71 | 72 | } 73 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/Kafka09AvroTableSink.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 | package com.datafibers.flinknext; 19 | 20 | import java.util.Properties; 21 | 22 | import com.datafibers.util.ConstantApp; 23 | import com.datafibers.util.SchemaRegistryClient; 24 | import org.apache.flink.api.java.tuple.Tuple2; 25 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 26 | import org.apache.flink.api.java.typeutils.TupleTypeInfo; 27 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; 28 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; 29 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; 30 | import org.apache.flink.streaming.util.serialization.SerializationSchema; 31 | import org.apache.flink.table.api.Types; 32 | import org.apache.flink.types.Row; 33 | 34 | /** 35 | * Kafka 0.9 {@link KafkaTableSink} that serializes data in JSON format. 36 | */ 37 | public class Kafka09AvroTableSink extends KafkaAvroTableSink { 38 | /** 39 | * Creates {@link KafkaTableSink} for Kafka 0.9 40 | * 41 | * @param topic topic in Kafka to which table is written 42 | * @param properties properties to connect to Kafka 43 | * @param partitioner Kafka partitioner 44 | */ 45 | 46 | private Boolean isAppendOnly; 47 | 48 | public Kafka09AvroTableSink(String topic, Properties properties, FlinkKafkaPartitioner> partitioner) { 49 | super(topic, properties, partitioner); 50 | } 51 | 52 | @Override 53 | protected FlinkKafkaProducerBase> createKafkaProducer(String topic, Properties properties, SerializationSchema> serializationSchema, FlinkKafkaPartitioner> partitioner) { 54 | return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner); 55 | } 56 | 57 | @Override 58 | protected Kafka09AvroTableSink createCopy() { 59 | return new Kafka09AvroTableSink(topic, properties, partitioner); 60 | } 61 | 62 | 63 | @Override 64 | protected SerializationSchema> createSerializationSchema(Properties properties) { 65 | // TODO Auto-generated method stub 66 | return new AvroRowSerializationSchema(properties); 67 | } 68 | 69 | @Override 70 | public void setKeyFields(String[] keys) { 71 | 72 | } 73 | 74 | @Override 75 | public void setIsAppendOnly(Boolean isAppendOnly) { 76 | this.isAppendOnly = isAppendOnly; 77 | } 78 | 79 | @Override 80 | public TupleTypeInfo> getOutputType() { 81 | return new TupleTypeInfo(Types.BOOLEAN(), 82 | new RowTypeInfo(SchemaRegistryClient.getFieldTypesInfoFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT))); 83 | } 84 | } -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/Kafka09AvroTableSource.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.flinknext; 2 | 3 | import java.util.Properties; 4 | 5 | import org.apache.flink.api.common.typeinfo.TypeInformation; 6 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; 7 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; 8 | import org.apache.flink.streaming.util.serialization.DeserializationSchema; 9 | import org.apache.flink.table.sources.StreamTableSource; 10 | import org.apache.flink.types.Row; 11 | /** 12 | * Kafka {@link StreamTableSource} for Kafka 0.9. 13 | */ 14 | public class Kafka09AvroTableSource extends KafkaAvroTableSource { 15 | 16 | /** 17 | * Creates a Kafka 0.9 AVRO {@link StreamTableSource}. 18 | * 19 | * @param topic Kafka topic to consume. 20 | * @param properties Properties for the Kafka consumer. 21 | * @param fieldNames Row field names. 22 | * @param fieldTypes Row field types. 23 | */ 24 | public Kafka09AvroTableSource( 25 | String topic, 26 | Properties properties, 27 | String[] fieldNames, 28 | TypeInformation[] fieldTypes) { 29 | 30 | super(topic, properties, fieldNames, fieldTypes); 31 | } 32 | 33 | /** 34 | * Creates a Kafka 0.9 AVRO fields name and type are derived from schema info in properties 35 | * @param topic - topic in Kafka to map the table source 36 | * @param properties - list of properties to connect Kafka, etc 37 | * 38 | *

Following property need to set ahead of using. 39 | *

properties.setProperty("schema.subject", "test-value"); // Subject Name for the SchemaRegistry 40 | *

properties.setProperty("schema.registry", "localhost:8081"); // Host and port for the SchemaRegistry 41 | *

properties.setProperty("static.avro.schema", STATIC_USER_SCHEMA); // Schema string when Schema is static. With Static Schema, SchemaRegistry does not have to be used. 42 | */ 43 | public Kafka09AvroTableSource(String topic, Properties properties) { 44 | 45 | super(topic, properties); 46 | } 47 | 48 | /** 49 | * Creates a Kafka 0.9 AVRO {@link StreamTableSource}. 50 | * 51 | * @param topic Kafka topic to consume. 52 | * @param properties Properties for the Kafka consumer. 53 | * @param fieldNames Row field names. 54 | * @param fieldTypes Row field types. 55 | */ 56 | public Kafka09AvroTableSource( 57 | String topic, 58 | Properties properties, 59 | String[] fieldNames, 60 | Class[] fieldTypes) { 61 | 62 | super(topic, properties, fieldNames, fieldTypes); 63 | } 64 | 65 | @Override 66 | FlinkKafkaConsumerBase getKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { 67 | return new FlinkKafkaConsumer09<>(topic, deserializationSchema, properties); 68 | } 69 | 70 | 71 | } 72 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/KafkaAvroTableSink.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 | package com.datafibers.flinknext; 19 | 20 | 21 | import java.util.Properties; 22 | 23 | import org.apache.flink.api.java.tuple.Tuple2; 24 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; 25 | import org.apache.flink.streaming.util.serialization.SerializationSchema; 26 | import org.apache.flink.types.Row; 27 | 28 | /** 29 | * Base class for {@link KafkaTableSink} that serializes data in JSON format 30 | */ 31 | public abstract class KafkaAvroTableSink extends KafkaTableSink { 32 | 33 | /** 34 | * Creates KafkaJsonTableSink 35 | * 36 | * @param topic topic in Kafka to which table is written 37 | * @param properties properties to connect to Kafka 38 | * @param partitioner Kafka partitioner 39 | */ 40 | public KafkaAvroTableSink(String topic, Properties properties, FlinkKafkaPartitioner> partitioner) { 41 | super(topic, properties, partitioner); 42 | } 43 | 44 | protected SerializationSchema> createSerializationSchema(Properties properties) { 45 | return new AvroRowSerializationSchema(properties); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/KafkaAvroTableSource.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.flinknext; 2 | 3 | import java.util.Properties; 4 | 5 | import com.datafibers.util.ConstantApp; 6 | import org.apache.flink.api.common.typeinfo.TypeInformation; 7 | import org.apache.flink.streaming.util.serialization.DeserializationSchema; 8 | import org.apache.flink.table.sources.StreamTableSource; 9 | 10 | import com.datafibers.util.SchemaRegistryClient; 11 | 12 | /** 13 | * A version-agnostic Kafka AVRO {@link StreamTableSource}. 14 | * 15 | *

The version-specific Kafka consumers need to extend this class and 16 | * override {@link #getKafkaConsumer(String, Properties, DeserializationSchema)}}. 17 | * 18 | *

The field names are used to parse the AVRO file and so are the types. 19 | */ 20 | public abstract class KafkaAvroTableSource extends KafkaTableSource { 21 | 22 | /** 23 | * Creates a generic Kafka AVRO {@link StreamTableSource}. 24 | * 25 | * @param topic Kafka topic to consume. 26 | * @param properties Properties for the Kafka consumer. 27 | * @param fieldNames Row field names. 28 | * @param fieldTypes Row field types. 29 | */ 30 | KafkaAvroTableSource( 31 | String topic, 32 | Properties properties, 33 | String[] fieldNames, 34 | Class[] fieldTypes) { 35 | 36 | super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes, properties), fieldNames, fieldTypes); 37 | } 38 | 39 | /** 40 | * Creates a generic Kafka AVRO with fields and types derived from Schema Registry 41 | * @param topic 42 | * @param properties 43 | */ 44 | KafkaAvroTableSource(String topic, Properties properties) { 45 | super(topic, properties, 46 | createDeserializationSchema( 47 | SchemaRegistryClient.getFieldNamesFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT), 48 | SchemaRegistryClient.getFieldTypesFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT), 49 | properties 50 | ), 51 | SchemaRegistryClient.getFieldNamesFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT), 52 | SchemaRegistryClient.getFieldTypesFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT) 53 | ); 54 | } 55 | 56 | /** 57 | * Creates a generic Kafka AVRO {@link StreamTableSource}. 58 | * 59 | * @param topic Kafka topic to consume. 60 | * @param properties Properties for the Kafka consumer. 61 | * @param fieldNames Row field names. 62 | * @param fieldTypes Row field types. 63 | */ 64 | KafkaAvroTableSource( 65 | String topic, 66 | Properties properties, 67 | String[] fieldNames, 68 | TypeInformation[] fieldTypes) { 69 | 70 | super(topic, properties, createDeserializationSchema(fieldNames, fieldTypes, properties), fieldNames, fieldTypes); 71 | } 72 | 73 | /** 74 | * Configures the failure behaviour if a AVRO field is missing. 75 | * 76 | *

By default, a missing field is ignored and the field is set to null. 77 | * 78 | * @param failOnMissingField Flag indicating whether to fail or not on a missing field. 79 | */ 80 | public void setFailOnMissingField(boolean failOnMissingField) { 81 | AvroRowDeserializationSchema deserializationSchema = (AvroRowDeserializationSchema) getDeserializationSchema(); 82 | deserializationSchema.setFailOnMissingField(failOnMissingField); 83 | } 84 | 85 | private static AvroRowDeserializationSchema createDeserializationSchema( 86 | String[] fieldNames, 87 | TypeInformation[] fieldTypes, Properties properties) { 88 | 89 | return new AvroRowDeserializationSchema(fieldNames, fieldTypes, properties); 90 | } 91 | 92 | private static AvroRowDeserializationSchema createDeserializationSchema( 93 | String[] fieldNames, 94 | Class[] fieldTypes, Properties properties) { 95 | 96 | return new AvroRowDeserializationSchema(fieldNames, fieldTypes, properties); 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/flinknext/KafkaTableSink.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 | package com.datafibers.flinknext; 19 | 20 | import java.util.Properties; 21 | 22 | import com.datafibers.util.ConstantApp; 23 | import org.apache.avro.Schema; 24 | import org.apache.flink.api.common.typeinfo.TypeInformation; 25 | import org.apache.flink.api.java.tuple.Tuple2; 26 | import org.apache.flink.api.java.typeutils.RowTypeInfo; 27 | import org.apache.flink.streaming.api.datastream.DataStream; 28 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; 29 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; 30 | import org.apache.flink.streaming.util.serialization.SerializationSchema; 31 | import org.apache.flink.table.sinks.UpsertStreamTableSink; 32 | import org.apache.flink.types.Row; 33 | import org.apache.flink.util.Preconditions; 34 | 35 | import com.datafibers.util.SchemaRegistryClient; 36 | 37 | /** 38 | * A version-agnostic Kafka {@link UpsertStreamTableSink}. 39 | * 40 | *

The version-specific Kafka consumers need to extend this class and 41 | * override {@link #createKafkaProducer(String, Properties, SerializationSchema, FlinkKafkaPartitioner)}}. 42 | */ 43 | public abstract class KafkaTableSink implements UpsertStreamTableSink { 44 | 45 | protected final String topic; 46 | protected final Properties properties; 47 | protected SerializationSchema> serializationSchema; 48 | protected final FlinkKafkaPartitioner> partitioner; 49 | protected String[] fieldNames; 50 | protected TypeInformation[] fieldTypes; 51 | protected Schema schema; 52 | /** 53 | * Creates KafkaTableSink 54 | * 55 | * @param topic Kafka topic to write to. 56 | * @param properties Properties for the Kafka consumer. 57 | * @param partitioner Partitioner to select Kafka partition for each item 58 | */ 59 | public KafkaTableSink( 60 | String topic, 61 | Properties properties, 62 | FlinkKafkaPartitioner> partitioner) { 63 | 64 | this.topic = Preconditions.checkNotNull(topic, "topic"); 65 | this.properties = Preconditions.checkNotNull(properties, "properties"); 66 | this.partitioner = Preconditions.checkNotNull(partitioner, "partitioner"); 67 | this.schema = SchemaRegistryClient.getLatestSchemaFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT); 68 | //setIsAppendOnly(false); 69 | setIsAppendOnly(true); 70 | setKeyFields(properties.getProperty(ConstantApp.PK_FLINK_TABLE_SINK_KEYS).split(",")); 71 | } 72 | 73 | /** 74 | * Returns the version-specifid Kafka producer. 75 | * 76 | * @param topic Kafka topic to produce to. 77 | * @param properties Properties for the Kafka producer. 78 | * @param serializationSchema Serialization schema to use to create Kafka records. 79 | * @param partitioner Partitioner to select Kafka partition. 80 | * @return The version-specific Kafka producer 81 | */ 82 | protected abstract FlinkKafkaProducerBase> createKafkaProducer( 83 | String topic, Properties properties, 84 | SerializationSchema> serializationSchema, 85 | FlinkKafkaPartitioner> partitioner); 86 | 87 | /** 88 | * Create serialization schema for converting table rows into bytes. 89 | * 90 | * @param properties 91 | * @return 92 | */ 93 | 94 | protected abstract SerializationSchema> createSerializationSchema(Properties properties) ; 95 | 96 | /** 97 | * Create a deep copy of this sink. 98 | * 99 | * @return Deep copy of this sink 100 | */ 101 | protected abstract KafkaTableSink createCopy(); 102 | 103 | @Override 104 | public void emitDataStream(DataStream> dataStream) { 105 | dataStream.addSink(createKafkaProducer(topic, properties, serializationSchema, partitioner)); 106 | } 107 | 108 | public String[] getFieldNames() { 109 | return fieldNames; 110 | } 111 | 112 | //@Override 113 | public TypeInformation[] getFieldTypes() { 114 | return fieldTypes; 115 | } 116 | 117 | @Override 118 | public TypeInformation getRecordType() { 119 | return new RowTypeInfo(fieldTypes); 120 | } 121 | 122 | @Override 123 | public KafkaTableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { 124 | KafkaTableSink copy = createCopy(); 125 | copy.fieldNames = SchemaRegistryClient.getFieldNamesFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT); 126 | copy.fieldTypes = SchemaRegistryClient.getFieldTypesInfoFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT); 127 | copy.serializationSchema = createSerializationSchema(properties); 128 | return copy; 129 | } 130 | 131 | } 132 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/model/DFLogPOPJ.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.model; 2 | 3 | import io.vertx.core.json.JsonObject; 4 | 5 | /** 6 | * Created by will on 2017-08-11. 7 | */ 8 | public class DFLogPOPJ { 9 | 10 | private String id; // id as pk, which is also used as task id 11 | private String timestamp; // Identify each task order in a job 12 | private String level; // Name of the task 13 | private String message; 14 | private String fileName; 15 | private String method; 16 | private String lineNumber; 17 | private String className; 18 | 19 | public DFLogPOPJ(String id, String timestamp, String level, String className, String fileName, String lineNumber, String method, String message) { 20 | this.id = id; 21 | this.timestamp = timestamp; 22 | this.level = level; 23 | this.className = className; 24 | this.fileName = fileName; 25 | this.lineNumber = lineNumber; 26 | this.method = method; 27 | this.message = message; 28 | } 29 | 30 | // Used by 31 | public DFLogPOPJ(JsonObject json) { 32 | this.id = json.getJsonObject("_id").getString("$oid"); 33 | this.timestamp = json.getJsonObject("timestamp").getString("$date"); 34 | this.level = json.getString("level"); 35 | this.className = json.getJsonObject("class").getString("className"); 36 | this.fileName = json.getString("fileName"); 37 | this.lineNumber = json.getString("lineNumber"); 38 | this.method = json.getString("method"); 39 | this.message = json.getString("message"); 40 | 41 | } 42 | 43 | public String getId() { 44 | return id; 45 | } 46 | 47 | public void setId(String id) { 48 | this.id = id; 49 | } 50 | 51 | public String getTimestamp() { 52 | return timestamp; 53 | } 54 | 55 | public void setTimestamp(String timestamp) { 56 | this.timestamp = timestamp; 57 | } 58 | 59 | public String getLevel() { 60 | return level; 61 | } 62 | 63 | public void setLevel(String level) { 64 | this.level = level; 65 | } 66 | 67 | public String getClassName() { 68 | return className; 69 | } 70 | 71 | public void setClassName(String className) { 72 | this.className = className; 73 | } 74 | 75 | public String getFileName() { 76 | return fileName; 77 | } 78 | 79 | public void setFileName(String fileName) { 80 | this.fileName = fileName; 81 | } 82 | 83 | public String getLineNumber() { 84 | return lineNumber; 85 | } 86 | 87 | public void setLineNumber(String lineNumber) { 88 | this.lineNumber = lineNumber; 89 | } 90 | 91 | public String getMethod() { 92 | return method; 93 | } 94 | 95 | public void setMethod(String method) { 96 | this.method = method; 97 | } 98 | 99 | public String getMessage() { 100 | return message; 101 | } 102 | 103 | public void setMessage(String message) { 104 | this.message = message; 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/service/DFInitService.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.service; 2 | 3 | import com.datafibers.util.CLIParser; 4 | import com.datafibers.util.MongoAdminClient; 5 | import com.datafibers.util.Runner; 6 | import com.datafibers.test_tool.UnitTestSuiteFlink; 7 | import org.apache.commons.codec.DecoderException; 8 | import org.apache.commons.lang3.StringUtils; 9 | import org.apache.log4j.Logger; 10 | 11 | import java.io.IOException; 12 | 13 | public class DFInitService { 14 | 15 | private static final Logger LOG = Logger.getLogger(DFInitService.class); 16 | private static String runningMode; 17 | 18 | public static void main(String[] args) { 19 | 20 | welcome(); 21 | LOG.info("********* DataFibers Services is starting. *********"); 22 | 23 | CLIParser cli = new CLIParser(args); 24 | cli.parse(); 25 | runningMode = cli.getRunMode(); 26 | 27 | if (runningMode == null || runningMode.contains("DEBUG")) { 28 | Runner.runExample(DFDataProcessor.class); 29 | Runner.runExample(DFWebUI.class); 30 | } else { 31 | if (runningMode.contains("ADMIN_TOOL")) runAdminTools(); 32 | if (runningMode.contains("TEST")) runTestCases(); 33 | if (runningMode.contains("Cluster")) Runner.runClusteredExample(DFDataProcessor.class); 34 | if (runningMode.contains("Standalone")) Runner.runExample(DFDataProcessor.class); 35 | //if (runningMode.contains("WebUI")) Runner.runExample(DFWebUI.class); 36 | } 37 | } 38 | 39 | public static void welcome() { 40 | System.out.println(" __ __ _ _ ___ _ ___ _ _ \n" + 41 | "/ / /\\ \\ \\___| | ___ ___ _ __ ___ ___ | |_ ___ / \\__ _| |_ __ _ / __(_) |__ ___ _ __ ___ \n" + 42 | "\\ \\/ \\/ / _ \\ |/ __/ _ \\| '_ ` _ \\ / _ \\ | __/ _ \\ / /\\ / _` | __/ _` |/ _\\ | | '_ \\ / _ \\ '__/ __|\n" + 43 | " \\ /\\ / __/ | (_| (_) | | | | | | __/ | || (_) | / /_// (_| | || (_| / / | | |_) | __/ | \\__ \\\n" + 44 | " \\/ \\/ \\___|_|\\___\\___/|_| |_| |_|\\___| \\__\\___/ /___,' \\__,_|\\__\\__,_\\/ |_|_.__/ \\___|_| |___/\n" + 45 | " "); 46 | } 47 | 48 | public static void runTestCases() { 49 | String testcaseNumber = runningMode.replaceAll("[^0-9]", ""); 50 | switch (testcaseNumber) { 51 | case "1": 52 | case "2": 53 | UnitTestSuiteFlink.testFlinkAvroSQL(); 54 | break; 55 | case "4": 56 | UnitTestSuiteFlink.testFlinkAvroSQLJson(); 57 | case "5": 58 | UnitTestSuiteFlink.testFlinkAvroSQLWithStaticSchema(); 59 | case "6": 60 | UnitTestSuiteFlink.testFlinkAvroScriptWithStaticSchema(); 61 | default: 62 | break; 63 | } 64 | } 65 | 66 | public static void runAdminTools() { 67 | String adminTool = StringUtils.substringAfterLast(runningMode, "ADMIN_TOOL_"); 68 | if (adminTool.equalsIgnoreCase("remove_tasks")) { 69 | LOG.info("Clean up all tasks (except internal) from repo at localhost:27017/DEFAULT_DB/df_processor"); 70 | new MongoAdminClient("localhost", 27017, "DEFAULT_DB") 71 | .truncateCollectionExcept("df_processor", "connectorCategory", "INTERNAL") 72 | .close(); 73 | } 74 | 75 | if (adminTool.contains("remove_tasks(")) { 76 | String[] para = StringUtils.substringBetween(adminTool, "(", ")").split(","); 77 | LOG.info("Clean up all tasks (except internal) from repo at " 78 | + para[0] + ":" + para[1] + "/" + para[2] + "/" + para[3]); 79 | new MongoAdminClient(para[0], Integer.parseInt(para[1]), para[2]) 80 | .truncateCollectionExcept(para[3], "connectorCategory", "INTERNAL") 81 | .close(); 82 | 83 | } 84 | 85 | if (adminTool.equalsIgnoreCase("import_df_install") || 86 | adminTool.equalsIgnoreCase("idi")) { 87 | LOG.info("Import Connect Metadata to repo at localhost:27017/DEFAULT_DB/df_installed"); 88 | new MongoAdminClient("localhost", "27017", "DEFAULT_DB", "df_installed") 89 | .truncateCollection("df_installed") 90 | .importJsonInputStream(DFInitService.class.getResourceAsStream("/import/df_installed.json")) 91 | .close(); 92 | } 93 | 94 | if (adminTool.contains("import_df_install(") || adminTool.contains("idi(")) { 95 | String[] para = StringUtils.substringBetween(adminTool, "(", ")").split(","); 96 | LOG.info("Clean up all tasks (except internal) from repo at " 97 | + para[0] + ":" + para[1] + "/" + para[2] + "/" + para[3]); 98 | new MongoAdminClient(para[0], para[1], para[2], para[3]) 99 | .truncateCollection(para[3]) 100 | .importJsonInputStream(DFInitService.class.getResourceAsStream("/import/df_installed.json")) 101 | .close(); 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/service/DFWebUI.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.service; 2 | 3 | import com.datafibers.util.DFAPIMessage; 4 | import io.vertx.core.AbstractVerticle; 5 | import io.vertx.ext.web.Router; 6 | import io.vertx.ext.web.handler.StaticHandler; 7 | import org.apache.log4j.Logger; 8 | import java.net.InetAddress; 9 | import java.net.UnknownHostException; 10 | 11 | public class DFWebUI extends AbstractVerticle { 12 | 13 | private static final Logger LOG = Logger.getLogger(DFWebUI.class); 14 | 15 | @Override 16 | public void start() { 17 | 18 | // Create a router object for web ui 19 | Router routerWeb = Router.router(vertx); 20 | 21 | // Bind new web ui 22 | routerWeb.route("/dfa/*").handler(StaticHandler.create("dfa").setCachingEnabled(false)); 23 | // Bind api doc 24 | routerWeb.route("/api/*").handler(StaticHandler.create("apidoc").setCachingEnabled(true)); 25 | // Bind landing page 26 | routerWeb.route("/*").handler(StaticHandler.create("landing").setCachingEnabled(true)); 27 | 28 | // Create the HTTP server to serve the web ui 29 | vertx.createHttpServer().requestHandler(routerWeb::accept) 30 | .listen(config().getInteger("http.port.df.processor", 8000)); 31 | 32 | try { 33 | LOG.info("DataFibers Welcome You @ http://" + InetAddress.getLocalHost().getHostAddress() + ":" + 34 | config().getInteger("http.port.df.processor", 8000)); 35 | } catch (UnknownHostException e) { 36 | LOG.error(DFAPIMessage.logResponseMessage(9019, 37 | "NetworkHostException - " + e.getCause())); 38 | } 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/AvroConsumerTest.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | 3 | import io.confluent.kafka.serializers.KafkaAvroDeserializer; 4 | import org.apache.kafka.clients.consumer.*; 5 | import org.apache.kafka.common.TopicPartition; 6 | 7 | import java.time.Instant; 8 | import java.util.*; 9 | 10 | import static java.time.temporal.ChronoUnit.MINUTES; 11 | import static java.time.temporal.ChronoUnit.SECONDS; 12 | 13 | /** 14 | * Created by will on 2017-09-12. 15 | */ 16 | public class AvroConsumerTest { 17 | 18 | private static final Properties props = new Properties(); 19 | 20 | static { 21 | props.put("bootstrap.servers", "localhost:9092"); 22 | props.put("group.id", "group1"); 23 | props.put("schema.registry.url", "http://localhost:8002"); 24 | props.put("enable.auto.commit", "true"); 25 | props.put("auto.commit.interval.ms", "1000"); 26 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaAvroDeserializer.class); 27 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaAvroDeserializer.class); 28 | } 29 | 30 | public static void consumeAll(String topic) { 31 | KafkaConsumer consumer = new KafkaConsumer<>(props); 32 | consumer.subscribe(Arrays.asList(topic)); 33 | while (true) { 34 | ConsumerRecords records = consumer.poll(100); 35 | for (ConsumerRecord record : records) 36 | System.out.printf("offset = %d, key = %s, value = %s%n", record.offset(), record.key(), record.value()); 37 | if (!records.isEmpty()) { 38 | consumer.commitSync(); 39 | break; 40 | } 41 | } 42 | } 43 | 44 | public static void consumeFromTime(String topic) { 45 | KafkaConsumer consumer = new KafkaConsumer<>(props); 46 | consumer.subscribe(Arrays.asList(topic)); 47 | 48 | boolean flag = true; 49 | 50 | while (true) { 51 | ConsumerRecords records = consumer.poll(100); 52 | 53 | if (flag) { 54 | Set assignments = consumer.assignment(); 55 | Map query = new HashMap<>(); 56 | for (TopicPartition topicPartition : assignments) { 57 | query.put( 58 | topicPartition, 59 | Instant.now().minus(5, MINUTES).toEpochMilli()); 60 | } 61 | 62 | Map result = consumer.offsetsForTimes(query); 63 | 64 | result.entrySet() 65 | .stream() 66 | .forEach(entry -> 67 | consumer.seek( 68 | entry.getKey(), 69 | Optional.ofNullable(entry.getValue()) 70 | .map(OffsetAndTimestamp::offset) 71 | .orElse(new Long(0)))); 72 | 73 | flag = false; 74 | } 75 | 76 | for (ConsumerRecord record : records) 77 | System.out.printf("offset = %d, key = %s, value = %s%n", record.offset(), record.key(), record.value()); 78 | if (!records.isEmpty()) break; 79 | } 80 | 81 | 82 | } 83 | 84 | public static void consumeBatch(String topic) { 85 | KafkaConsumer consumer = new KafkaConsumer<>(props); 86 | consumer.subscribe(Arrays.asList(topic)); 87 | final int minBatchSize = 10; 88 | List> buffer = new ArrayList<>(); 89 | while (true) { 90 | ConsumerRecords records = consumer.poll(100); 91 | for (ConsumerRecord record : records) { 92 | buffer.add(record); 93 | } 94 | if (buffer.size() >= minBatchSize) { 95 | consumer.commitSync(); 96 | buffer.forEach(System.out::println); 97 | buffer.clear(); 98 | break; 99 | } 100 | } 101 | } 102 | 103 | public static void main(String[] args) { 104 | AvroConsumerTest.consumeFromTime("test_stock"); 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/AvroConsumerVertx.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | 3 | import io.confluent.kafka.serializers.KafkaAvroDeserializer; 4 | import io.vertx.core.AbstractVerticle; 5 | import io.vertx.core.DeploymentOptions; 6 | import io.vertx.core.Vertx; 7 | import io.vertx.core.json.JsonObject; 8 | import io.vertx.kafka.client.common.PartitionInfo; 9 | import io.vertx.kafka.client.consumer.KafkaConsumer; 10 | import org.apache.kafka.clients.consumer.ConsumerConfig; 11 | 12 | import java.util.ArrayList; 13 | import java.util.Properties; 14 | 15 | 16 | /** 17 | * @author Julien Viet 18 | */ 19 | public class AvroConsumerVertx extends AbstractVerticle { 20 | 21 | public static void main(String[] args) { 22 | Vertx vertx = Vertx.vertx(); 23 | vertx.deployVerticle(new AvroConsumerVertx()); 24 | } 25 | 26 | 27 | @Override 28 | public void start() throws Exception { 29 | System.out.println("Test"); 30 | Properties props = new Properties(); 31 | props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 32 | props.put(ConsumerConfig.GROUP_ID_CONFIG, "group1"); 33 | props.put("schema.registry.url", "http://localhost:8002"); 34 | props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); 35 | props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "1000"); 36 | props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, KafkaAvroDeserializer.class); 37 | props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, KafkaAvroDeserializer.class); 38 | String topic = "test_stock"; 39 | 40 | KafkaConsumer consumer = KafkaConsumer.create(vertx, props); 41 | ArrayList responseList = new ArrayList(); 42 | 43 | // consumer.handler(record -> {// TODO handler does not work 44 | // System.out.println("Processing value=" + record.record().value() + 45 | // ",partition=" + record.record().partition() + ",offset=" + record.record().offset()); 46 | // responseList.add(new JsonObject() 47 | // .put("offset", record.record().offset()) 48 | // .put("value", record.record().value().toString())); 49 | // if(responseList.size() >= 10 ) { 50 | // consumer.pause(); 51 | // consumer.commit(); 52 | // consumer.close(); 53 | // } 54 | // }); 55 | // 56 | // // Subscribe to a single topic 57 | // consumer.subscribe(topic, ar -> { 58 | // if (ar.succeeded()) { 59 | // System.out.println("topic " + topic + " is subscribed"); 60 | // } else { 61 | // System.out.println("Could not subscribe " + ar.cause().getMessage()); 62 | // } 63 | // }); 64 | 65 | consumer.partitionsFor(topic, ar -> { 66 | 67 | if (ar.succeeded()) { 68 | 69 | for (PartitionInfo partitionInfo : ar.result()) { 70 | System.out.println(partitionInfo); 71 | } 72 | } 73 | }); 74 | 75 | } 76 | 77 | @Override 78 | public void stop() throws Exception { 79 | 80 | } 81 | } 82 | 83 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/CodeGenFlinkTable.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | import net.openhft.compiler.CompilerUtils; 3 | 4 | import org.apache.flink.api.common.typeinfo.TypeInformation; 5 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 6 | import org.apache.flink.table.api.Table; 7 | import org.apache.flink.table.api.TableEnvironment; 8 | import org.apache.flink.table.api.Types; 9 | import org.apache.flink.table.api.java.StreamTableEnvironment; 10 | import org.apache.flink.table.sinks.CsvTableSink; 11 | import org.apache.flink.table.sinks.TableSink; 12 | import org.apache.flink.table.sources.CsvTableSource; 13 | 14 | import com.datafibers.util.DynamicRunner; 15 | 16 | /** 17 | * The goal is to define a function with parameter TABLE, Transformation Script and return TABLE 18 | * This function will dynamically generate code and run. 19 | */ 20 | public class CodeGenFlinkTable { 21 | 22 | public static Table getFlinkTableObj(String className, String javaCode){ 23 | try { 24 | Class aClass = CompilerUtils.CACHED_COMPILER.loadFromJava(className, javaCode); 25 | DynamicRunner runner = (DynamicRunner) aClass.newInstance(); 26 | //return runner.getTableObj(); 27 | 28 | } catch (Exception e) { 29 | e.printStackTrace(); 30 | } 31 | return null; 32 | } 33 | 34 | public static void main(String args[]) { 35 | 36 | String transform = "flatMap(new FlinkUDF.LineSplitter()).groupBy(0).sum(1).print();\n"; 37 | 38 | String transform2 = "select(\"name\");\n"; 39 | 40 | String header = "package dynamic;\n" + 41 | "import org.apache.flink.api.table.Table;\n" + 42 | "import com.datafibers.util.*;\n"; 43 | 44 | String javaCode = header + 45 | "public class FlinkScript implements DynamicRunner {\n" + 46 | "@Override \n" + 47 | " public void runTransform(DataSet ds) {\n" + 48 | "try {" + 49 | "ds."+ transform + 50 | "} catch (Exception e) {" + 51 | "};" + 52 | "}}"; 53 | 54 | String javaCode2 = header + 55 | "public class FlinkScript implements DynamicRunner {\n" + 56 | "@Override \n" + 57 | " public Table transTableObj(Table tbl) {\n" + 58 | "try {" + 59 | "return tbl."+ transform2 + 60 | "} catch (Exception e) {" + 61 | "};" + 62 | "return null;}}"; 63 | 64 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 65 | StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); 66 | CsvTableSource csvTableSource = new CsvTableSource( 67 | "/Users/will/Downloads/file.csv", 68 | new String[] { "name", "id", "score", "comments" }, 69 | new TypeInformation[] { 70 | Types.STRING(), 71 | Types.STRING(), 72 | Types.STRING(), 73 | Types.STRING() 74 | }); // lenient 75 | 76 | tableEnv.registerTableSource("mycsv", csvTableSource); 77 | TableSink sink = new CsvTableSink("/Users/will/Downloads/out.csv", "|"); 78 | Table ingest = tableEnv.scan("mycsv"); 79 | 80 | try { 81 | String className = "dynamic.FlinkScript"; 82 | Class aClass = CompilerUtils.CACHED_COMPILER.loadFromJava(className, javaCode2); 83 | DynamicRunner runner = (DynamicRunner) aClass.newInstance(); 84 | //runner.runTransform(ds); 85 | Table result = runner.transTableObj(ingest); 86 | // write the result Table to the TableSink 87 | result.writeToSink(sink); 88 | env.execute(); 89 | 90 | } catch (Exception e) { 91 | e.printStackTrace(); 92 | } 93 | } 94 | } -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/FlinkUDF.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | 3 | import org.apache.flink.api.common.functions.FlatMapFunction; 4 | import org.apache.flink.api.java.DataSet; 5 | import org.apache.flink.api.java.ExecutionEnvironment; 6 | import org.apache.flink.api.java.tuple.Tuple2; 7 | import org.apache.flink.util.Collector; 8 | 9 | public class FlinkUDF { 10 | 11 | /** 12 | * Implements the string tokenizer that splits sentences into words as a user-defined 13 | * FlatMapFunction. The function takes a line (String) and splits it into 14 | * multiple pairs in the form of "(word,1)" (Tuple2). 15 | */ 16 | public static final class LineSplitter implements FlatMapFunction> { 17 | 18 | @Override 19 | public void flatMap(String value, Collector> out) { 20 | // normalize and split the line 21 | String[] tokens = value.toLowerCase().split("\\W+"); 22 | 23 | // emit the pairs 24 | for (String token : tokens) { 25 | if (token.length() > 0) { 26 | out.collect(new Tuple2(token, 1)); 27 | } 28 | } 29 | } 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/JsonTest.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | import com.datafibers.service.DFInitService; 3 | import com.datafibers.util.ConstantApp; 4 | import com.datafibers.util.HelpFunc; 5 | import com.datafibers.util.MongoAdminClient; 6 | import io.vertx.core.json.JsonArray; 7 | import io.vertx.core.json.JsonObject; 8 | import org.apache.commons.codec.DecoderException; 9 | import org.apache.commons.configuration.SystemConfiguration; 10 | import org.apache.commons.lang3.StringUtils; 11 | 12 | import java.io.IOException; 13 | import java.util.Arrays; 14 | import java.util.List; 15 | import java.util.regex.Pattern; 16 | 17 | /** 18 | * Created by DUW3 on 11/11/2016. 19 | */ 20 | public class JsonTest { 21 | 22 | public static String arrayToString(JsonArray ja) { 23 | String result = ""; 24 | for (int i = 0; i < ja.size(); i++) { 25 | result = result + ja.getValue(i).toString() + ","; 26 | } 27 | return result.substring(0, result.length() - 1); 28 | } 29 | 30 | public static JsonArray livyTableResultToArray(JsonObject livyStatementResult) { 31 | JsonObject output = livyStatementResult 32 | .getJsonObject("output") 33 | .getJsonObject("data") 34 | .getJsonObject("application/vnd.livy.table.v1+json"); 35 | 36 | JsonArray header = output.getJsonArray("headers"); 37 | JsonArray data = output.getJsonArray("data"); 38 | JsonArray result = new JsonArray(); 39 | JsonObject headerRowJson = new JsonObject(); 40 | String headerRow = ""; 41 | 42 | if(header.size() == 0) return new JsonArray().add(new JsonObject().put("row", "")); 43 | 44 | for(int i = 0; i < header.size(); i++) { 45 | headerRow = headerRow + header.getJsonObject(i).getString("name") + ","; 46 | } 47 | 48 | result.add(headerRowJson.put("row", headerRow)); 49 | 50 | for(int i = 0; i < data.size(); i++) { 51 | result.add(new JsonObject().put("row", arrayToString(data.getJsonArray(i)))); 52 | } 53 | 54 | return result; 55 | } 56 | 57 | public static void main(String[] args) throws IOException, DecoderException { 58 | String mySb = "--comments \n" + 59 | "select * from test;--comments2\n" + 60 | "select * from\n" + 61 | "test2;"; 62 | mySb = "--comments \n" + 63 | "select * from test;"; 64 | Arrays.asList(HelpFunc.sqlCleaner(mySb)).forEach(System.out::println); 65 | 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/KafkaClientTest.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | 3 | import com.datafibers.util.KafkaAdminClient; 4 | 5 | public class KafkaClientTest { 6 | 7 | public static void main(String[] args) { 8 | KafkaAdminClient.listTopics("localhost:9092") 9 | .forEach(System.out::println); 10 | 11 | KafkaAdminClient.createTopic("localhost:9092", "test", 1, 1); 12 | 13 | KafkaAdminClient.listTopics("localhost:9092") 14 | .forEach(System.out::println); 15 | //new KafkaAdminClient().deleteTopics("my_topic"); 16 | //System.out.println(new KafkaAdminClient().existsTopic("df_meta")); 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/SparkStructuredStreamingDemo.java: -------------------------------------------------------------------------------- 1 | /* 2 | package com.datafibers.test_tool; 3 | 4 | import com.databricks.spark.avro.SchemaConverters; 5 | import com.twitter.bijection.Injection; 6 | import com.twitter.bijection.avro.GenericAvroCodecs; 7 | import org.apache.avro.Schema; 8 | import org.apache.avro.generic.GenericRecord; 9 | import org.apache.log4j.Level; 10 | import org.apache.log4j.LogManager; 11 | import org.apache.spark.SparkConf; 12 | import org.apache.spark.sql.Dataset; 13 | import org.apache.spark.sql.Row; 14 | import org.apache.spark.sql.RowFactory; 15 | import org.apache.spark.sql.SparkSession; 16 | import org.apache.spark.sql.streaming.StreamingQuery; 17 | import org.apache.spark.sql.streaming.StreamingQueryException; 18 | import org.apache.spark.sql.types.DataTypes; 19 | import org.apache.spark.sql.types.StructType; 20 | 21 | */ 22 | /** 23 | * Structured streaming demo using Avro'ed Kafka topic as input 24 | *//* 25 | 26 | public class SparkStructuredStreamingDemo { 27 | 28 | private static Injection recordInjection; 29 | private static StructType type; 30 | private static final String USER_SCHEMA = "{" 31 | + "\"type\":\"record\"," 32 | + "\"name\":\"myrecord\"," 33 | + "\"fields\":[" 34 | + " { \"name\":\"company_name\", \"type\":\"string\" }," 35 | + " { \"name\":\"symbol\", \"type\":\"string\" }," 36 | + " { \"name\":\"exchange\", \"type\":\"string\" }," 37 | + " { \"name\":\"ask_size\", \"type\":\"int\" }" 38 | + "]}"; 39 | private static Schema.Parser parser = new Schema.Parser(); 40 | private static Schema schema = parser.parse(USER_SCHEMA); 41 | private static int schemaSize = schema.getFields().size(); 42 | private static Object[] records = new Object[schemaSize]; 43 | 44 | static { //once per VM, lazily 45 | recordInjection = GenericAvroCodecs.toBinary(schema); 46 | type = (StructType) SchemaConverters.toSqlType(schema).dataType(); 47 | } 48 | 49 | public static void main(String[] args) throws StreamingQueryException { 50 | //set log4j programmatically 51 | LogManager.getLogger("org.apache.spark").setLevel(Level.WARN); 52 | LogManager.getLogger("akka").setLevel(Level.ERROR); 53 | 54 | //configure Spark 55 | SparkConf conf = new SparkConf() 56 | .setAppName("kafka-structured") 57 | .setMaster("local[*]"); 58 | 59 | //initialize spark session 60 | SparkSession sparkSession = SparkSession 61 | .builder() 62 | .config(conf) 63 | .getOrCreate(); 64 | 65 | //reduce task number 66 | sparkSession.sqlContext().setConf("spark.sql.shuffle.partitions", "3"); 67 | 68 | //data stream from kafka 69 | Dataset ds1 = sparkSession 70 | .readStream() 71 | .format("kafka") 72 | .option("kafka.bootstrap.servers", "localhost:9092") 73 | .option("subscribe", "test_spark2") 74 | .option("startingOffsets", "earliest") 75 | .load(); 76 | 77 | //start the streaming query 78 | sparkSession.udf().register("deserialize", (byte[] data) -> { 79 | byte[] transplant = new byte[data.length - 5]; 80 | System.arraycopy(data, 5, transplant, 0, data.length - 5); 81 | GenericRecord record = recordInjection.invert(transplant).get(); 82 | for(int i = 0; i < schemaSize; i ++) { 83 | if(schema.getFields().get(i).schema().getType().toString().equalsIgnoreCase("STRING")) { 84 | records[i] = record.get(schema.getFields().get(i).name()).toString(); 85 | } else { 86 | records[i] = record.get(schema.getFields().get(i).name()); 87 | } 88 | } 89 | return RowFactory.create(records); 90 | 91 | }, DataTypes.createStructType(type.fields())); 92 | ds1.printSchema(); 93 | 94 | Dataset ds2 = ds1.selectExpr("cast(key as string) as keys", "deserialize(value) as rows") 95 | .select("rows.*"); 96 | 97 | ds2.printSchema(); 98 | 99 | StreamingQuery query1 = ds2 100 | .writeStream() 101 | .queryName("Test query") 102 | //.outputMode("complete") 103 | .format("console") 104 | .start(); 105 | 106 | query1.awaitTermination(); 107 | 108 | } 109 | } 110 | */ 111 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/UploadTest.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | 3 | import com.datafibers.util.HelpFunc; 4 | 5 | /** 6 | * Created by dadu on 24/10/2017. 7 | */ 8 | public class UploadTest { 9 | 10 | public static void main(String[] args) { 11 | 12 | String jar_id = HelpFunc.uploadJar("http://localhost:8001/jars/upload", 13 | "C:/Users/dadu/Coding/df_data_service/target/df-data-service-1.1-SNAPSHOT-fat.jar"); 14 | System.out.println(jar_id); 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/test_tool/WordCountStream.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.test_tool; 2 | import org.apache.flink.api.common.typeinfo.TypeInformation; 3 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 4 | import org.apache.flink.table.api.Table; 5 | import org.apache.flink.table.api.TableEnvironment; 6 | import org.apache.flink.table.api.Types; 7 | import org.apache.flink.table.api.java.StreamTableEnvironment; 8 | import org.apache.flink.table.sinks.CsvTableSink; 9 | import org.apache.flink.table.sinks.TableSink; 10 | import org.apache.flink.table.sources.CsvTableSource; 11 | 12 | /** 13 | * The goal is to define a function with parameter TABLE, Transformation Script and return TABLE 14 | * This function will dynamically generate code and run. 15 | */ 16 | public class WordCountStream { 17 | 18 | 19 | public static void main(String args[]) { 20 | 21 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 22 | StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); 23 | 24 | // Create a DataStream from a list of elements 25 | //DataStream ds = env.fromElements(1, 2, 3, 4, 5); 26 | 27 | CsvTableSource csvTableSource = new CsvTableSource( 28 | "/Users/will/Downloads/file.csv", 29 | new String[] { "name", "id", "score", "comments" }, 30 | new TypeInformation[] { 31 | Types.STRING(), 32 | Types.STRING(), 33 | Types.STRING(), 34 | Types.STRING() 35 | }); // lenient 36 | 37 | tableEnv.registerTableSource("mycsv", csvTableSource); 38 | 39 | 40 | 41 | TableSink sink = new CsvTableSink("/Users/will/Downloads/out.csv", "|"); 42 | 43 | 44 | //tableEnv.registerDataStream("tbl", ds, "a"); 45 | //Table ingest = tableEnv.fromDataStream(ds, "name"); 46 | Table in = tableEnv.scan("mycsv"); 47 | //Table in = tableEnv.ingest("tbl"); 48 | //Table in = tableEnv.fromDataStream(ds, "a"); 49 | 50 | Table result = in.select("name"); 51 | result.writeToSink(sink); 52 | try { 53 | env.execute(); 54 | } catch (Exception e) { 55 | 56 | } 57 | 58 | System.out.print("DONE"); 59 | } 60 | } -------------------------------------------------------------------------------- /src/main/java/com/datafibers/util/CLIParser.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.util; 2 | 3 | import com.datafibers.processor.*; 4 | import com.datafibers.service.DFDataProcessor; 5 | import com.datafibers.service.DFInitService; 6 | import com.datafibers.service.DFWebUI; 7 | import org.apache.commons.cli.*; 8 | import org.apache.log4j.Level; 9 | import org.apache.log4j.LogManager; 10 | import org.apache.log4j.Logger; 11 | 12 | public class CLIParser { 13 | private static final Logger LOG = Logger.getLogger(CLIParser.class); 14 | private String[] args = null; 15 | private Options options = new Options(); 16 | public static String run_mode = ""; 17 | public static String service_mode = ""; 18 | public static String test_mode = ""; 19 | public static String admin_tool = ""; 20 | public static String debug_mode = ""; 21 | 22 | 23 | public CLIParser(String[] args) { 24 | this.args = args; 25 | options.addOption("h", "help", false, "show usage help"); 26 | options.addOption("d", "debug", false, "run application in debug level"); 27 | options.addOption("t", "test", true, "run configured test cases, =test_case_number"); 28 | options.addOption("u", "webui", true, "enable web, =ui|noui"); 29 | options.addOption("m", "mode", true, "running vertx mode, =cluster|standalone"); 30 | options.addOption("a", "admin", true, 31 | "run admin tools, =Function, such as " + 32 | "\n Function: remove_tasks - remove all tasks/processors from repo." + 33 | "\n Usage: " + 34 | "\n -a remove_tasks" + 35 | "\n -a remove_tasks(localhost,27017,db_name,db_collection_name)" + 36 | "\n Function: import_df_install (aka. idi) - rebuild df_install configs" + 37 | "\n Usage: " + 38 | "\n -a import_df_install" + 39 | "\n -a import_df_install(localhost,27017,db_name,db_collection_name)" 40 | ); 41 | } 42 | 43 | public CommandLine parse() { 44 | CommandLineParser parser = new DefaultParser(); 45 | CommandLine cmd = null; 46 | if (args == null || args.length == 0) return null; 47 | try { 48 | cmd = parser.parse(options, args); 49 | if (cmd.hasOption("h")) 50 | help(); 51 | 52 | if (cmd.hasOption("d")) { 53 | this.debug_mode = "DEBUG"; 54 | LogManager.getLogger(DFInitService.class).setLevel(Level.DEBUG); 55 | LogManager.getLogger(DFWebUI.class).setLevel(Level.DEBUG); 56 | LogManager.getLogger(DFDataProcessor.class).setLevel(Level.DEBUG); 57 | LogManager.getLogger(ProcessorConnectKafka.class).setLevel(Level.DEBUG); 58 | LogManager.getLogger(ProcessorStreamBack.class).setLevel(Level.DEBUG); 59 | LogManager.getLogger(ProcessorTopicSchemaRegistry.class).setLevel(Level.DEBUG); 60 | LogManager.getLogger(ProcessorTransformFlink.class).setLevel(Level.DEBUG); 61 | LogManager.getLogger(ProcessorTransformSpark.class).setLevel(Level.DEBUG); 62 | } 63 | 64 | if (cmd.hasOption("m")) { 65 | if(cmd.getOptionValue("m").equalsIgnoreCase("cluster")) { 66 | this.run_mode = "Cluster"; // Cluster 67 | } else { 68 | this.run_mode = "Standalone"; // Standalone 69 | } 70 | // Whatever you want to do with the setting goes here 71 | } 72 | 73 | if (cmd.hasOption("u")) { 74 | if(cmd.getOptionValue("u").equalsIgnoreCase("ui")) { 75 | this.service_mode = "WebUI"; // UI only 76 | } else { 77 | this.service_mode = "Processor"; // Processor Only 78 | } 79 | } 80 | 81 | if (cmd.hasOption("t")) { 82 | if(cmd.getOptionValue("t").matches("[-+]?\\d*\\.?\\d+")) { 83 | this.test_mode = "TEST_CASE_" + cmd.getOptionValue("t"); 84 | } else { 85 | this.test_mode = "TEST_CASE_1"; 86 | } 87 | } 88 | 89 | if (cmd.hasOption("a")) { 90 | if(cmd.getOptionValue("a") != null) 91 | this.admin_tool = "ADMIN_TOOL_" + cmd.getOptionValue("a"); 92 | } 93 | 94 | } catch (ParseException e) { 95 | LOG.warn(DFAPIMessage.logResponseMessage(9020, "exception - " + e.getCause())); 96 | help(); 97 | } 98 | return null; 99 | } 100 | 101 | public String getRunMode () { 102 | if (args == null || args.length == 0) { 103 | LOG.info("Starting both DataFibers Service and Web UI ..."); 104 | return null; 105 | } 106 | 107 | if(args.length > 0 && args[0].contains("-conf")) // ignore -conf option which is used by vertx config 108 | return null; 109 | 110 | LOG.info("Starting DataFibers in customized options."); 111 | LOG.info("run_mode = " + this.run_mode); 112 | LOG.info("service_mode = " + this.service_mode); 113 | LOG.info("test_mode = " + this.test_mode); 114 | LOG.info("admin_tool = " + this.admin_tool); 115 | return this.run_mode + this.service_mode + this.test_mode + this.admin_tool + this.debug_mode; 116 | } 117 | 118 | public void help() { 119 | // This prints out some help 120 | HelpFormatter formatter = new HelpFormatter(); 121 | formatter.printHelp("Main", options); 122 | System.exit(0); 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/util/DynamicRunner.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.util; 2 | 3 | import org.apache.flink.api.java.DataSet; 4 | import org.apache.flink.table.api.Table; 5 | import org.apache.flink.streaming.api.datastream.DataStream; 6 | 7 | /** 8 | * Interface for dynamic Flink stable api script generation and running 9 | */ 10 | public interface DynamicRunner { 11 | 12 | /* 13 | * Run Flink Table API Transformation 14 | */ 15 | default void runTransform(DataSet ds) { 16 | 17 | } 18 | 19 | default void runTransform(DataStream ds) { 20 | 21 | } 22 | 23 | default Table transTableObj(Table tbl) { 24 | return tbl; 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/util/FlinkAvroSQLClient.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.util; 2 | 3 | import com.datafibers.flinknext.Kafka010AvroTableSource; 4 | import com.datafibers.flinknext.Kafka09AvroTableSink; 5 | import org.apache.commons.codec.DecoderException; 6 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 7 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; 8 | import org.apache.flink.table.api.Table; 9 | import org.apache.flink.table.api.TableEnvironment; 10 | import org.apache.flink.table.api.java.StreamTableEnvironment; 11 | import java.io.IOException; 12 | import java.util.Properties; 13 | 14 | /** 15 | * Flink Client to Submit SQL job through Flink Rest API 16 | */ 17 | public class FlinkAvroSQLClient { 18 | 19 | public static void tcFlinkAvroSQL(String KafkaServerHostPort, String SchemaRegistryHostPort, 20 | String srcTopic, String targetTopic, 21 | String consumerGroupId, String sinkKeys, String sqlState) { 22 | 23 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 24 | StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); 25 | 26 | Properties properties = new Properties(); 27 | properties.setProperty(ConstantApp.PK_KAFKA_HOST_PORT.replace("_", "."), KafkaServerHostPort); 28 | properties.setProperty(ConstantApp.PK_KAFKA_CONSUMER_GROURP, consumerGroupId); 29 | properties.setProperty(ConstantApp.PK_KAFKA_SCHEMA_REGISTRY_HOST_PORT.replace("_", "."), SchemaRegistryHostPort); 30 | properties.setProperty(ConstantApp.PK_FLINK_TABLE_SINK_KEYS, sinkKeys); 31 | 32 | String[] srcTopicList = srcTopic.split(","); 33 | for (int i = 0; i < srcTopicList.length; i++) { 34 | properties.setProperty(ConstantApp.PK_SCHEMA_SUB_INPUT, srcTopicList[i]); 35 | properties.setProperty(ConstantApp.PK_SCHEMA_ID_INPUT, SchemaRegistryClient.getLatestSchemaIDFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT) + ""); 36 | properties.setProperty(ConstantApp.PK_SCHEMA_STR_INPUT, SchemaRegistryClient.getLatestSchemaFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT).toString()); 37 | tableEnv.registerTableSource(srcTopicList[i], new Kafka010AvroTableSource(srcTopicList[i], properties)); 38 | } 39 | 40 | try { 41 | Table result = tableEnv.sql(sqlState); 42 | SchemaRegistryClient.addSchemaFromTableResult(SchemaRegistryHostPort, targetTopic, result); 43 | // For old producer, we need to create topic-value subject as well 44 | SchemaRegistryClient.addSchemaFromTableResult(SchemaRegistryHostPort, targetTopic + "-value", result); 45 | 46 | // delivered properties for sink 47 | properties.setProperty(ConstantApp.PK_SCHEMA_SUB_OUTPUT, targetTopic); 48 | properties.setProperty(ConstantApp.PK_SCHEMA_ID_OUTPUT, SchemaRegistryClient.getLatestSchemaIDFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT) + ""); 49 | properties.setProperty(ConstantApp.PK_SCHEMA_STR_OUTPUT, SchemaRegistryClient.getLatestSchemaFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT).toString()); 50 | 51 | Kafka09AvroTableSink avro_sink = 52 | new Kafka09AvroTableSink(targetTopic, properties, new FlinkFixedPartitioner()); 53 | result.writeToSink(avro_sink); 54 | env.execute("DF_FlinkSQL_Client_" + srcTopic + "-" + targetTopic); 55 | } catch (Exception e) { 56 | e.printStackTrace(); 57 | } 58 | } 59 | 60 | public static void main(String[] args) throws IOException, DecoderException { 61 | //tcFlinkAvroSQL("localhost:9092", "localhost:8081", "test_stock", "SQLSTATE_UNION_01", "symbol", "consumergroupid", SQLSTATE_UNION_01); 62 | tcFlinkAvroSQL(args[0], args[1], args[2], args[3], args[4], args[5], args[6]); 63 | 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/util/FlinkAvroTableAPIClient.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.util; 2 | 3 | import com.datafibers.flinknext.Kafka010AvroTableSource; 4 | import com.datafibers.flinknext.Kafka09AvroTableSink; 5 | import net.openhft.compiler.CompilerUtils; 6 | import org.apache.commons.codec.DecoderException; 7 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 8 | import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; 9 | import org.apache.flink.table.api.Table; 10 | import org.apache.flink.table.api.TableEnvironment; 11 | import org.apache.flink.table.api.java.StreamTableEnvironment; 12 | 13 | import java.io.IOException; 14 | import java.util.Properties; 15 | 16 | /** 17 | * Flink Client to Submit Table API job through Flink Rest API 18 | */ 19 | public class FlinkAvroTableAPIClient { 20 | 21 | public static void tcFlinkAvroTableAPI(String KafkaServerHostPort, String SchemaRegistryHostPort, 22 | String srcTopic, String targetTopic, 23 | String consumerGroupId, String sinkKeys, String transScript) { 24 | 25 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 26 | StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); 27 | 28 | Properties properties = new Properties(); 29 | properties.setProperty(ConstantApp.PK_KAFKA_HOST_PORT.replace("_", "."), KafkaServerHostPort); 30 | properties.setProperty(ConstantApp.PK_KAFKA_CONSUMER_GROURP, consumerGroupId); 31 | properties.setProperty(ConstantApp.PK_KAFKA_SCHEMA_REGISTRY_HOST_PORT.replace("_", "."), SchemaRegistryHostPort); 32 | properties.setProperty(ConstantApp.PK_FLINK_TABLE_SINK_KEYS, sinkKeys); 33 | 34 | String[] srcTopicList = srcTopic.split(","); 35 | for (int i = 0; i < srcTopicList.length; i++) { 36 | properties.setProperty(ConstantApp.PK_SCHEMA_SUB_INPUT, srcTopicList[i]); 37 | properties.setProperty(ConstantApp.PK_SCHEMA_ID_INPUT, SchemaRegistryClient.getLatestSchemaIDFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT) + ""); 38 | properties.setProperty(ConstantApp.PK_SCHEMA_STR_INPUT, SchemaRegistryClient.getLatestSchemaFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_INPUT).toString()); 39 | tableEnv.registerTableSource(srcTopic, new Kafka010AvroTableSource(srcTopicList[i], properties)); 40 | } 41 | 42 | try { 43 | Table result; 44 | Table ingest = tableEnv.scan(srcTopic); 45 | String className = "dynamic.FlinkScript"; 46 | String header = "package dynamic;\n" + 47 | "import org.apache.flink.table.api.Table;\n" + 48 | "import com.datafibers.util.*;\n"; 49 | String javaCode = header + 50 | "public class FlinkScript implements DynamicRunner {\n" + 51 | "@Override \n" + 52 | " public Table transTableObj(Table tbl) {\n" + 53 | "try {" + 54 | "return tbl." + transScript + ";\n" + 55 | "} catch (Exception e) {" + 56 | "};" + 57 | "return null;}}"; 58 | // Dynamic code generation 59 | Class aClass = CompilerUtils.CACHED_COMPILER.loadFromJava(className, javaCode); 60 | DynamicRunner runner = (DynamicRunner) aClass.newInstance(); 61 | result = runner.transTableObj(ingest); 62 | 63 | SchemaRegistryClient.addSchemaFromTableResult(SchemaRegistryHostPort, targetTopic, result); 64 | // delivered properties for sink 65 | properties.setProperty(ConstantApp.PK_SCHEMA_SUB_OUTPUT, targetTopic); 66 | properties.setProperty(ConstantApp.PK_SCHEMA_ID_OUTPUT, SchemaRegistryClient.getLatestSchemaIDFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT) + ""); 67 | properties.setProperty(ConstantApp.PK_SCHEMA_STR_OUTPUT, SchemaRegistryClient.getLatestSchemaFromProperty(properties, ConstantApp.PK_SCHEMA_SUB_OUTPUT).toString()); 68 | 69 | Kafka09AvroTableSink avro_sink = 70 | new Kafka09AvroTableSink(targetTopic, properties, new FlinkFixedPartitioner()); 71 | result.writeToSink(avro_sink); 72 | env.execute("DF_FlinkTableAPI_Client_" + srcTopic + "-" + targetTopic); 73 | } catch (Exception e) { 74 | e.printStackTrace(); 75 | } 76 | } 77 | 78 | public static void main(String[] args) throws IOException, DecoderException { 79 | //tcFlinkAvroSQL("localhost:9092", "localhost:8081", "test_stock", "APISTATE_UNION_01", "symbol", "consumergroupid", SQLSTATE_UNION_01); 80 | tcFlinkAvroTableAPI(args[0], args[1], args[2], args[3], args[4], args[5], args[6]); 81 | 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /src/main/java/com/datafibers/util/MongoAdminClient.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.util; 2 | 3 | import com.mongodb.MongoClient; 4 | import com.mongodb.client.MongoCollection; 5 | import com.mongodb.client.MongoCursor; 6 | import com.mongodb.client.MongoDatabase; 7 | import com.mongodb.client.MongoIterable; 8 | import static com.mongodb.client.model.Filters.ne; 9 | import static com.mongodb.client.model.Filters.eq; 10 | import static com.mongodb.client.model.Filters.exists; 11 | import com.mongodb.client.model.BulkWriteOptions; 12 | import com.mongodb.client.model.InsertOneModel; 13 | import io.vertx.core.json.JsonObject; 14 | import org.bson.Document; 15 | 16 | import java.io.*; 17 | import java.util.ArrayList; 18 | import java.util.List; 19 | 20 | /** 21 | * MongoDB Client 22 | */ 23 | public class MongoAdminClient { 24 | private MongoClient mongoClient; 25 | private MongoDatabase database; 26 | private MongoCollection collection; 27 | 28 | 29 | public MongoAdminClient(String hostname, int port, String database) { 30 | this.mongoClient = new MongoClient(hostname, port ); 31 | this.database = this.mongoClient.getDatabase(database); 32 | } 33 | 34 | public MongoAdminClient(String hostname, String port, String database) { 35 | this.mongoClient = new MongoClient(hostname, Integer.parseInt(port)); 36 | this.database = this.mongoClient.getDatabase(database); 37 | } 38 | 39 | public MongoAdminClient(String hostname, String port, String database, String collection) { 40 | this.mongoClient = new MongoClient(hostname, Integer.parseInt(port)); 41 | this.database = this.mongoClient.getDatabase(database); 42 | this.collection = this.database.getCollection(collection); 43 | } 44 | 45 | public MongoAdminClient truncateCollection(String colName) { 46 | if(collectionExists(colName)) 47 | this.database.getCollection(colName).deleteMany(exists("_id")); //TODO consider to add date filter 48 | return this; 49 | } 50 | 51 | public MongoAdminClient truncateCollectionExcept(String colName, String key, String value) { 52 | if(collectionExists(colName)) 53 | this.database.getCollection(colName).deleteMany(ne(key, value)); 54 | return this; 55 | } 56 | 57 | public MongoAdminClient dropCollection(String colName) { 58 | if(collectionExists(colName)) 59 | this.database.getCollection(colName).drop(); 60 | return this; 61 | } 62 | 63 | public MongoAdminClient createCollection(String colName) { 64 | if(!collectionExists(colName)) 65 | this.database.createCollection(colName); 66 | return this; 67 | } 68 | 69 | public MongoAdminClient useCollection(String colName) { 70 | if(collectionExists(colName)) 71 | this.collection = this.database.getCollection(colName); 72 | return this; 73 | } 74 | 75 | public boolean collectionExists(String collectionName) { 76 | if (this.database == null) { 77 | return false; 78 | } 79 | 80 | final MongoIterable iterable = database.listCollectionNames(); 81 | try (final MongoCursor it = iterable.iterator()) { 82 | while (it.hasNext()) { 83 | if (it.next().equalsIgnoreCase(collectionName)) { 84 | return true; 85 | } 86 | } 87 | } 88 | 89 | return false; 90 | } 91 | 92 | /** 93 | * Lookup A collection using specific key and value and return value from another value 94 | * @param lkpKey 95 | * @param lkpValue 96 | * @param lkpReturnedKey 97 | * @return lkpReturnedKValue 98 | */ 99 | public String lkpCollection(String lkpKey, String lkpValue, String lkpReturnedKey) { 100 | String connectorClass = this.collection.find(eq(lkpKey, lkpValue)).first().toJson(); 101 | return new JsonObject(connectorClass).getString(lkpReturnedKey); 102 | } 103 | 104 | public MongoAdminClient importJsonFile(String fileNamePath) { 105 | int count = 0; 106 | int batch = 100; 107 | 108 | List> docs = new ArrayList<>(); 109 | 110 | try (BufferedReader br = new BufferedReader(new FileReader(fileNamePath))) { 111 | String line; 112 | while ((line = br.readLine()) != null) { 113 | docs.add(new InsertOneModel<>(Document.parse(line))); 114 | count++; 115 | if (count == batch) { 116 | this.collection.bulkWrite(docs, new BulkWriteOptions().ordered(false)); 117 | docs.clear(); 118 | count = 0; 119 | } 120 | } 121 | } catch (IOException fnfe) { 122 | fnfe.printStackTrace(); 123 | } 124 | 125 | if (count > 0) { 126 | collection.bulkWrite(docs, new BulkWriteOptions().ordered(false)); 127 | } 128 | 129 | return this; 130 | } 131 | 132 | public MongoAdminClient importJsonInputStream(InputStream fileInputStream) { 133 | int count = 0; 134 | int batch = 100; 135 | 136 | List> docs = new ArrayList<>(); 137 | 138 | try (BufferedReader br = new BufferedReader(new InputStreamReader(fileInputStream))) { 139 | String line; 140 | while ((line = br.readLine()) != null) { 141 | docs.add(new InsertOneModel<>(Document.parse(line))); 142 | count++; 143 | if (count == batch) { 144 | this.collection.bulkWrite(docs, new BulkWriteOptions().ordered(false)); 145 | docs.clear(); 146 | count = 0; 147 | } 148 | } 149 | } catch (IOException fnfe) { 150 | fnfe.printStackTrace(); 151 | } 152 | 153 | if (count > 0) { 154 | collection.bulkWrite(docs, new BulkWriteOptions().ordered(false)); 155 | } 156 | 157 | return this; 158 | } 159 | 160 | public void close() { 161 | this.mongoClient.close(); 162 | } 163 | } 164 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/api_project.js: -------------------------------------------------------------------------------- 1 | define({ 2 | "name": "Welcome DataFibers Data Service API", 3 | "version": "0.1.1", 4 | "description": "DataFibers REST Service Specification.", 5 | "title": "DataFibers REST Service Specification", 6 | "url": "http://localhost:8080/api/df", 7 | "header": { 8 | "title": "DF Guide", 9 | "content": "

Please refer DataFibers Complete Guide for more details.

\n" 10 | }, 11 | "footer": { 12 | "title": "DF Website", 13 | "content": "

Visit DataFibers' website or github.

\n" 14 | }, 15 | "order": [ 16 | "All", 17 | "Connect", 18 | "Transform", 19 | "Schema" 20 | ], 21 | "template": { 22 | "withCompare": true, 23 | "withGenerator": true 24 | }, 25 | "sampleUrl": false, 26 | "apidoc": "0.2.0", 27 | "generator": { 28 | "name": "apidoc", 29 | "time": "2016-12-29T16:02:00.415Z", 30 | "url": "http://apidocjs.com", 31 | "version": "0.16.1" 32 | } 33 | }); 34 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/api_project.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "Welcome DataFibers Data Service API", 3 | "version": "0.1.1", 4 | "description": "DataFibers REST Service Specification.", 5 | "title": "DataFibers REST Service Specification", 6 | "url": "http://localhost:8080/api/df", 7 | "header": { 8 | "title": "DF Guide", 9 | "content": "

Please refer DataFibers Complete Guide for more details.

\n" 10 | }, 11 | "footer": { 12 | "title": "DF Website", 13 | "content": "

Visit DataFibers' website or github.

\n" 14 | }, 15 | "order": [ 16 | "All", 17 | "Connect", 18 | "Transform", 19 | "Schema" 20 | ], 21 | "template": { 22 | "withCompare": true, 23 | "withGenerator": true 24 | }, 25 | "sampleUrl": false, 26 | "apidoc": "0.2.0", 27 | "generator": { 28 | "name": "apidoc", 29 | "time": "2016-12-29T16:02:00.415Z", 30 | "url": "http://apidocjs.com", 31 | "version": "0.16.1" 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/img/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/apidoc/img/favicon.ico -------------------------------------------------------------------------------- /src/main/resources/apidoc/img/glyphicons-halflings-white.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/apidoc/img/glyphicons-halflings-white.png -------------------------------------------------------------------------------- /src/main/resources/apidoc/img/glyphicons-halflings.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/apidoc/img/glyphicons-halflings.png -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/ca.js: -------------------------------------------------------------------------------- 1 | define({ 2 | ca: { 3 | 'Allowed values:' : 'Valors permesos:', 4 | 'Compare all with predecessor': 'Comparar tot amb versió anterior', 5 | 'compare changes to:' : 'comparar canvis amb:', 6 | 'compared to' : 'comparat amb', 7 | 'Default value:' : 'Valor per defecte:', 8 | 'Description' : 'Descripció', 9 | 'Field' : 'Camp', 10 | 'General' : 'General', 11 | 'Generated with' : 'Generat amb', 12 | 'Name' : 'Nom', 13 | 'No response values.' : 'Sense valors en la resposta.', 14 | 'optional' : 'opcional', 15 | 'Parameter' : 'Paràmetre', 16 | 'Permission:' : 'Permisos:', 17 | 'Response' : 'Resposta', 18 | 'Send' : 'Enviar', 19 | 'Send a Sample Request' : 'Enviar una petició d\'exemple', 20 | 'show up to version:' : 'mostrar versió:', 21 | 'Size range:' : 'Tamany de rang:', 22 | 'Type' : 'Tipus', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/de.js: -------------------------------------------------------------------------------- 1 | define({ 2 | de: { 3 | 'Allowed values:' : 'Erlaubte Werte:', 4 | 'Compare all with predecessor': 'Vergleiche alle mit ihren Vorgängern', 5 | 'compare changes to:' : 'vergleiche Änderungen mit:', 6 | 'compared to' : 'verglichen mit', 7 | 'Default value:' : 'Standardwert:', 8 | 'Description' : 'Beschreibung', 9 | 'Field' : 'Feld', 10 | 'General' : 'Allgemein', 11 | 'Generated with' : 'Erstellt mit', 12 | 'Name' : 'Name', 13 | 'No response values.' : 'Keine Rückgabewerte.', 14 | 'optional' : 'optional', 15 | 'Parameter' : 'Parameter', 16 | 'Permission:' : 'Berechtigung:', 17 | 'Response' : 'Antwort', 18 | 'Send' : 'Senden', 19 | 'Send a Sample Request' : 'Eine Beispielanfrage senden', 20 | 'show up to version:' : 'zeige bis zur Version:', 21 | 'Size range:' : 'Größenbereich:', 22 | 'Type' : 'Typ', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/es.js: -------------------------------------------------------------------------------- 1 | define({ 2 | es: { 3 | 'Allowed values:' : 'Valores permitidos:', 4 | 'Compare all with predecessor': 'Comparar todo con versión anterior', 5 | 'compare changes to:' : 'comparar cambios con:', 6 | 'compared to' : 'comparado con', 7 | 'Default value:' : 'Valor por defecto:', 8 | 'Description' : 'Descripción', 9 | 'Field' : 'Campo', 10 | 'General' : 'General', 11 | 'Generated with' : 'Generado con', 12 | 'Name' : 'Nombre', 13 | 'No response values.' : 'Sin valores en la respuesta.', 14 | 'optional' : 'opcional', 15 | 'Parameter' : 'Parámetro', 16 | 'Permission:' : 'Permisos:', 17 | 'Response' : 'Respuesta', 18 | 'Send' : 'Enviar', 19 | 'Send a Sample Request' : 'Enviar una petición de ejemplo', 20 | 'show up to version:' : 'mostrar a versión:', 21 | 'Size range:' : 'Tamaño de rango:', 22 | 'Type' : 'Tipo', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/fr.js: -------------------------------------------------------------------------------- 1 | define({ 2 | fr: { 3 | 'Allowed values:' : 'Valeurs autorisées :', 4 | 'Compare all with predecessor': 'Tout comparer avec ...', 5 | 'compare changes to:' : 'comparer les changements à :', 6 | 'compared to' : 'comparer à', 7 | 'Default value:' : 'Valeur par défaut :', 8 | 'Description' : 'Description', 9 | 'Field' : 'Champ', 10 | 'General' : 'Général', 11 | 'Generated with' : 'Généré avec', 12 | 'Name' : 'Nom', 13 | 'No response values.' : 'Aucune valeur de réponse.', 14 | 'optional' : 'optionnel', 15 | 'Parameter' : 'Paramètre', 16 | 'Permission:' : 'Permission :', 17 | 'Response' : 'Réponse', 18 | 'Send' : 'Envoyer', 19 | 'Send a Sample Request' : 'Envoyer une requête représentative', 20 | 'show up to version:' : 'Montrer à partir de la version :', 21 | 'Size range:' : 'Ordre de grandeur :', 22 | 'Type' : 'Type', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/it.js: -------------------------------------------------------------------------------- 1 | define({ 2 | it: { 3 | 'Allowed values:' : 'Valori permessi:', 4 | 'Compare all with predecessor': 'Confronta tutto con versioni precedenti', 5 | 'compare changes to:' : 'confronta modifiche con:', 6 | 'compared to' : 'confrontato con', 7 | 'Default value:' : 'Valore predefinito:', 8 | 'Description' : 'Descrizione', 9 | 'Field' : 'Campo', 10 | 'General' : 'Generale', 11 | 'Generated with' : 'Creato con', 12 | 'Name' : 'Nome', 13 | 'No response values.' : 'Nessnu valore di risposta.', 14 | 'optional' : 'opzionale', 15 | 'Parameter' : 'Parametro', 16 | 'Permission:' : 'Permessi:', 17 | 'Response' : 'Risposta', 18 | 'Send' : 'Invia', 19 | 'Send a Sample Request' : 'Invia una richiesta di esempio', 20 | 'show up to version:' : 'visualizza la versione:', 21 | 'Size range:' : 'Intervallo dimensione:', 22 | 'Type' : 'Tipo', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/locale.js: -------------------------------------------------------------------------------- 1 | define([ 2 | './locales/ca.js', 3 | './locales/de.js', 4 | './locales/es.js', 5 | './locales/fr.js', 6 | './locales/it.js', 7 | './locales/nl.js', 8 | './locales/pl.js', 9 | './locales/pt_br.js', 10 | './locales/ru.js', 11 | './locales/zh.js', 12 | './locales/zh_cn.js' 13 | ], function() { 14 | var langId = (navigator.language || navigator.userLanguage).toLowerCase().replace('-', '_'); 15 | var language = langId.substr(0, 2); 16 | var locales = {}; 17 | 18 | for (index in arguments) { 19 | for (property in arguments[index]) 20 | locales[property] = arguments[index][property]; 21 | } 22 | if ( ! locales['en']) 23 | locales['en'] = {}; 24 | 25 | if ( ! locales[langId] && ! locales[language]) 26 | language = 'en'; 27 | 28 | var locale = (locales[langId] ? locales[langId] : locales[language]); 29 | 30 | function __(text) { 31 | var index = locale[text]; 32 | if (index === undefined) 33 | return text; 34 | return index; 35 | }; 36 | 37 | function setLanguage(language) { 38 | locale = locales[language]; 39 | } 40 | 41 | return { 42 | __ : __, 43 | locales : locales, 44 | locale : locale, 45 | setLanguage: setLanguage 46 | }; 47 | }); 48 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/nl.js: -------------------------------------------------------------------------------- 1 | define({ 2 | nl: { 3 | 'Allowed values:' : 'Toegestane waarden:', 4 | 'Compare all with predecessor': 'Vergelijk alle met voorgaande versie', 5 | 'compare changes to:' : 'vergelijk veranderingen met:', 6 | 'compared to' : 'vergelijk met', 7 | 'Default value:' : 'Standaard waarde:', 8 | 'Description' : 'Omschrijving', 9 | 'Field' : 'Veld', 10 | 'General' : 'Algemeen', 11 | 'Generated with' : 'Gegenereerd met', 12 | 'Name' : 'Naam', 13 | 'No response values.' : 'Geen response waardes.', 14 | 'optional' : 'optioneel', 15 | 'Parameter' : 'Parameter', 16 | 'Permission:' : 'Permissie:', 17 | 'Response' : 'Antwoorden', 18 | 'Send' : 'Sturen', 19 | 'Send a Sample Request' : 'Stuur een sample aanvragen', 20 | 'show up to version:' : 'toon tot en met versie:', 21 | 'Size range:' : 'Maatbereik:', 22 | 'Type' : 'Type', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/pl.js: -------------------------------------------------------------------------------- 1 | define({ 2 | pl: { 3 | 'Allowed values:' : 'Dozwolone wartości:', 4 | 'Compare all with predecessor': 'Porównaj z poprzednimi wersjami', 5 | 'compare changes to:' : 'porównaj zmiany do:', 6 | 'compared to' : 'porównaj do:', 7 | 'Default value:' : 'Wartość domyślna:', 8 | 'Description' : 'Opis', 9 | 'Field' : 'Pole', 10 | 'General' : 'Generalnie', 11 | 'Generated with' : 'Wygenerowano z', 12 | 'Name' : 'Nazwa', 13 | 'No response values.' : 'Brak odpowiedzi.', 14 | 'optional' : 'opcjonalny', 15 | 'Parameter' : 'Parametr', 16 | 'Permission:' : 'Uprawnienia:', 17 | 'Response' : 'Odpowiedź', 18 | 'Send' : 'Wyślij', 19 | 'Send a Sample Request' : 'Wyślij przykładowe żądanie', 20 | 'show up to version:' : 'pokaż do wersji:', 21 | 'Size range:' : 'Zakres rozmiaru:', 22 | 'Type' : 'Typ', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/pt_br.js: -------------------------------------------------------------------------------- 1 | define({ 2 | 'pt_br': { 3 | 'Allowed values:' : 'Valores permitidos:', 4 | 'Compare all with predecessor': 'Compare todos com antecessores', 5 | 'compare changes to:' : 'comparar alterações com:', 6 | 'compared to' : 'comparado com', 7 | 'Default value:' : 'Valor padrão:', 8 | 'Description' : 'Descrição', 9 | 'Field' : 'Campo', 10 | 'General' : 'Geral', 11 | 'Generated with' : 'Gerado com', 12 | 'Name' : 'Nome', 13 | 'No response values.' : 'Sem valores de resposta.', 14 | 'optional' : 'opcional', 15 | 'Parameter' : 'Parâmetro', 16 | 'Permission:' : 'Permissão:', 17 | 'Response' : 'Resposta', 18 | 'Send' : 'Enviar', 19 | 'Send a Sample Request' : 'Enviar um Exemplo de Pedido', 20 | 'show up to version:' : 'aparecer para a versão:', 21 | 'Size range:' : 'Faixa de tamanho:', 22 | 'Type' : 'Tipo', 23 | 'url' : 'url' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/ru.js: -------------------------------------------------------------------------------- 1 | define({ 2 | ru: { 3 | 'Allowed values:' : 'Допустимые значения:', 4 | 'Compare all with predecessor': 'Сравнить с предыдущей версией', 5 | 'compare changes to:' : 'сравнить с:', 6 | 'compared to' : 'в сравнении с', 7 | 'Default value:' : 'По умолчанию:', 8 | 'Description' : 'Описание', 9 | 'Field' : 'Название', 10 | 'General' : 'Общая информация', 11 | 'Generated with' : 'Сгенерировано с помощью', 12 | 'Name' : 'Название', 13 | 'No response values.' : 'Нет значений для ответа.', 14 | 'optional' : 'необязательный', 15 | 'Parameter' : 'Параметр', 16 | 'Permission:' : 'Разрешено:', 17 | 'Response' : 'Ответ', 18 | 'Send' : 'Отправить', 19 | 'Send a Sample Request' : 'Отправить тестовый запрос', 20 | 'show up to version:' : 'показать версию:', 21 | 'Size range:' : 'Ограничения:', 22 | 'Type' : 'Тип', 23 | 'url' : 'URL' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/zh.js: -------------------------------------------------------------------------------- 1 | define({ 2 | zh: { 3 | 'Allowed values​​:' : '允許值:', 4 | 'Compare all with predecessor': '預先比較所有', 5 | 'compare changes to:' : '比較變更:', 6 | 'compared to' : '對比', 7 | 'Default value:' : '默認值:', 8 | 'Description' : '描述', 9 | 'Field' : '字段', 10 | 'General' : '概括', 11 | 'Generated with' : '生成工具', 12 | 'Name' : '名稱', 13 | 'No response values​​.' : '無對應資料.', 14 | 'optional' : '選項', 15 | 'Parameter' : '參數', 16 | 'Permission:' : '允許:', 17 | 'Response' : '回應', 18 | 'Send' : '發送', 19 | 'Send a Sample Request' : '發送試用需求', 20 | 'show up to version:' : '顯示到版本:', 21 | 'Size range:' : '尺寸範圍:', 22 | 'Type' : '類型', 23 | 'url' : '網址' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/locales/zh_cn.js: -------------------------------------------------------------------------------- 1 | define({ 2 | 'zh_cn': { 3 | 'Allowed values:' : '允许值:', 4 | 'Compare all with predecessor': '与所有较早的比较', 5 | 'compare changes to:' : '将当前版本与指定版本比较:', 6 | 'compared to' : '相比于', 7 | 'Default value:' : '默认值:', 8 | 'Description' : '描述', 9 | 'Field' : '字段', 10 | 'General' : '概要', 11 | 'Generated with' : '基于', 12 | 'Name' : '名称', 13 | 'No response values.' : '无返回值.', 14 | 'optional' : '可选', 15 | 'Parameter' : '参数', 16 | 'Permission:' : '权限:', 17 | 'Response' : '返回', 18 | 'Send' : '发送', 19 | 'Send a Sample Request' : '发送示例请求', 20 | 'show up to version:' : '显示到指定版本:', 21 | 'Size range:' : '取值范围:', 22 | 'Type' : '类型', 23 | 'url' : '网址' 24 | } 25 | }); 26 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/path-to-regexp/LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2014 Blake Embrey (hello@blakeembrey.com) 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in 13 | all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 21 | THE SOFTWARE. 22 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/path-to-regexp/index.js: -------------------------------------------------------------------------------- 1 | var isArray = Array.isArray || function (arr) { 2 | return Object.prototype.toString.call(arr) == '[object Array]'; 3 | }; 4 | 5 | /** 6 | * Expose `pathToRegexp`. 7 | */ 8 | // module.exports = pathToRegexp 9 | 10 | /** 11 | * The main path matching regexp utility. 12 | * 13 | * @type {RegExp} 14 | */ 15 | var PATH_REGEXP = new RegExp([ 16 | // Match escaped characters that would otherwise appear in future matches. 17 | // This allows the user to escape special characters that won't transform. 18 | '(\\\\.)', 19 | // Match Express-style parameters and un-named parameters with a prefix 20 | // and optional suffixes. Matches appear as: 21 | // 22 | // "/:test(\\d+)?" => ["/", "test", "\d+", undefined, "?"] 23 | // "/route(\\d+)" => [undefined, undefined, undefined, "\d+", undefined] 24 | '([\\/.])?(?:\\:(\\w+)(?:\\(((?:\\\\.|[^)])*)\\))?|\\(((?:\\\\.|[^)])*)\\))([+*?])?', 25 | // Match regexp special characters that are always escaped. 26 | '([.+*?=^!:${}()[\\]|\\/])' 27 | ].join('|'), 'g'); 28 | 29 | /** 30 | * Escape the capturing group by escaping special characters and meaning. 31 | * 32 | * @param {String} group 33 | * @return {String} 34 | */ 35 | function escapeGroup (group) { 36 | return group.replace(/([=!:$\/()])/g, '\\$1'); 37 | } 38 | 39 | /** 40 | * Attach the keys as a property of the regexp. 41 | * 42 | * @param {RegExp} re 43 | * @param {Array} keys 44 | * @return {RegExp} 45 | */ 46 | function attachKeys (re, keys) { 47 | re.keys = keys; 48 | return re; 49 | } 50 | 51 | /** 52 | * Get the flags for a regexp from the options. 53 | * 54 | * @param {Object} options 55 | * @return {String} 56 | */ 57 | function flags (options) { 58 | return options.sensitive ? '' : 'i'; 59 | } 60 | 61 | /** 62 | * Pull out keys from a regexp. 63 | * 64 | * @param {RegExp} path 65 | * @param {Array} keys 66 | * @return {RegExp} 67 | */ 68 | function regexpToRegexp (path, keys) { 69 | // Use a negative lookahead to match only capturing groups. 70 | var groups = path.source.match(/\((?!\?)/g); 71 | 72 | if (groups) { 73 | for (var i = 0; i < groups.length; i++) { 74 | keys.push({ 75 | name: i, 76 | delimiter: null, 77 | optional: false, 78 | repeat: false 79 | }); 80 | } 81 | } 82 | 83 | return attachKeys(path, keys); 84 | } 85 | 86 | /** 87 | * Transform an array into a regexp. 88 | * 89 | * @param {Array} path 90 | * @param {Array} keys 91 | * @param {Object} options 92 | * @return {RegExp} 93 | */ 94 | function arrayToRegexp (path, keys, options) { 95 | var parts = []; 96 | 97 | for (var i = 0; i < path.length; i++) { 98 | parts.push(pathToRegexp(path[i], keys, options).source); 99 | } 100 | 101 | var regexp = new RegExp('(?:' + parts.join('|') + ')', flags(options)); 102 | return attachKeys(regexp, keys); 103 | } 104 | 105 | /** 106 | * Replace the specific tags with regexp strings. 107 | * 108 | * @param {String} path 109 | * @param {Array} keys 110 | * @return {String} 111 | */ 112 | function replacePath (path, keys) { 113 | var index = 0; 114 | 115 | function replace (_, escaped, prefix, key, capture, group, suffix, escape) { 116 | if (escaped) { 117 | return escaped; 118 | } 119 | 120 | if (escape) { 121 | return '\\' + escape; 122 | } 123 | 124 | var repeat = suffix === '+' || suffix === '*'; 125 | var optional = suffix === '?' || suffix === '*'; 126 | 127 | keys.push({ 128 | name: key || index++, 129 | delimiter: prefix || '/', 130 | optional: optional, 131 | repeat: repeat 132 | }); 133 | 134 | prefix = prefix ? ('\\' + prefix) : ''; 135 | capture = escapeGroup(capture || group || '[^' + (prefix || '\\/') + ']+?'); 136 | 137 | if (repeat) { 138 | capture = capture + '(?:' + prefix + capture + ')*'; 139 | } 140 | 141 | if (optional) { 142 | return '(?:' + prefix + '(' + capture + '))?'; 143 | } 144 | 145 | // Basic parameter support. 146 | return prefix + '(' + capture + ')'; 147 | } 148 | 149 | return path.replace(PATH_REGEXP, replace); 150 | } 151 | 152 | /** 153 | * Normalize the given path string, returning a regular expression. 154 | * 155 | * An empty array can be passed in for the keys, which will hold the 156 | * placeholder key descriptions. For example, using `/user/:id`, `keys` will 157 | * contain `[{ name: 'id', delimiter: '/', optional: false, repeat: false }]`. 158 | * 159 | * @param {(String|RegExp|Array)} path 160 | * @param {Array} [keys] 161 | * @param {Object} [options] 162 | * @return {RegExp} 163 | */ 164 | function pathToRegexp (path, keys, options) { 165 | keys = keys || []; 166 | 167 | if (!isArray(keys)) { 168 | options = keys; 169 | keys = []; 170 | } else if (!options) { 171 | options = {}; 172 | } 173 | 174 | if (path instanceof RegExp) { 175 | return regexpToRegexp(path, keys, options); 176 | } 177 | 178 | if (isArray(path)) { 179 | return arrayToRegexp(path, keys, options); 180 | } 181 | 182 | var strict = options.strict; 183 | var end = options.end !== false; 184 | var route = replacePath(path, keys); 185 | var endsWithSlash = path.charAt(path.length - 1) === '/'; 186 | 187 | // In non-strict mode we allow a slash at the end of match. If the path to 188 | // match already ends with a slash, we remove it for consistency. The slash 189 | // is valid at the end of a path match, not in the middle. This is important 190 | // in non-ending mode, where "/test/" shouldn't match "/test//route". 191 | if (!strict) { 192 | route = (endsWithSlash ? route.slice(0, -2) : route) + '(?:\\/(?=$))?'; 193 | } 194 | 195 | if (end) { 196 | route += '$'; 197 | } else { 198 | // In non-ending mode, we need the capturing groups to match as much as 199 | // possible by using a positive lookahead to the end or next path segment. 200 | route += strict && endsWithSlash ? '' : '(?=\\/|$)'; 201 | } 202 | 203 | return attachKeys(new RegExp('^' + route, flags(options)), keys); 204 | } 205 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/polyfill.js: -------------------------------------------------------------------------------- 1 | // From https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Object/keys 2 | if (!Object.keys) { 3 | Object.keys = (function () { 4 | 'use strict'; 5 | var hasOwnProperty = Object.prototype.hasOwnProperty, 6 | hasDontEnumBug = !({toString: null}).propertyIsEnumerable('toString'), 7 | dontEnums = [ 8 | 'toString', 9 | 'toLocaleString', 10 | 'valueOf', 11 | 'hasOwnProperty', 12 | 'isPrototypeOf', 13 | 'propertyIsEnumerable', 14 | 'constructor' 15 | ], 16 | dontEnumsLength = dontEnums.length; 17 | 18 | return function (obj) { 19 | if (typeof obj !== 'object' && (typeof obj !== 'function' || obj === null)) { 20 | throw new TypeError('Object.keys called on non-object'); 21 | } 22 | 23 | var result = [], prop, i; 24 | 25 | for (prop in obj) { 26 | if (hasOwnProperty.call(obj, prop)) { 27 | result.push(prop); 28 | } 29 | } 30 | 31 | if (hasDontEnumBug) { 32 | for (i = 0; i < dontEnumsLength; i++) { 33 | if (hasOwnProperty.call(obj, dontEnums[i])) { 34 | result.push(dontEnums[i]); 35 | } 36 | } 37 | } 38 | return result; 39 | }; 40 | }()); 41 | } 42 | 43 | //Production steps of ECMA-262, Edition 5, 15.4.4.18 44 | //Reference: http://es5.github.com/#x15.4.4.18 45 | if (!Array.prototype.forEach) { 46 | 47 | Array.prototype.forEach = function (callback, thisArg) { 48 | 49 | var T, k; 50 | 51 | if (this == null) { 52 | throw new TypeError(" this is null or not defined"); 53 | } 54 | 55 | // 1. Let O be the result of calling ToObject passing the |this| value as the argument. 56 | var O = Object(this); 57 | 58 | // 2. Let lenValue be the result of calling the Get internal method of O with the argument "length". 59 | // 3. Let len be ToUint32(lenValue). 60 | var len = O.length >>> 0; 61 | 62 | // 4. If IsCallable(callback) is false, throw a TypeError exception. 63 | // See: http://es5.github.com/#x9.11 64 | if (typeof callback !== "function") { 65 | throw new TypeError(callback + " is not a function"); 66 | } 67 | 68 | // 5. If thisArg was supplied, let T be thisArg; else let T be undefined. 69 | if (arguments.length > 1) { 70 | T = thisArg; 71 | } 72 | 73 | // 6. Let k be 0 74 | k = 0; 75 | 76 | // 7. Repeat, while k < len 77 | while (k < len) { 78 | 79 | var kValue; 80 | 81 | // a. Let Pk be ToString(k). 82 | // This is implicit for LHS operands of the in operator 83 | // b. Let kPresent be the result of calling the HasProperty internal method of O with argument Pk. 84 | // This step can be combined with c 85 | // c. If kPresent is true, then 86 | if (k in O) { 87 | 88 | // i. Let kValue be the result of calling the Get internal method of O with argument Pk. 89 | kValue = O[k]; 90 | 91 | // ii. Call the Call internal method of callback with T as the this value and 92 | // argument list containing kValue, k, and O. 93 | callback.call(T, kValue, k, O); 94 | } 95 | // d. Increase k by 1. 96 | k++; 97 | } 98 | // 8. return undefined 99 | }; 100 | } 101 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify.css: -------------------------------------------------------------------------------- 1 | /* Pretty printing styles. Used with prettify.js. */ 2 | /* Vim sunburst theme by David Leibovic */ 3 | pre .str { 4 | color: #65B042; 5 | } 6 | /* string - green */ 7 | pre .kwd { 8 | color: #E28964; 9 | } 10 | /* keyword - dark pink */ 11 | pre .com { 12 | color: #AEAEAE; 13 | font-style: italic; 14 | } 15 | /* comment - gray */ 16 | pre .typ { 17 | color: #89bdff; 18 | } 19 | /* type - light blue */ 20 | pre .lit { 21 | color: #3387CC; 22 | } 23 | /* literal - blue */ 24 | pre .pun { 25 | color: #fff; 26 | } 27 | /* punctuation - white */ 28 | pre .pln { 29 | color: #fff; 30 | } 31 | /* plaintext - white */ 32 | pre .tag { 33 | color: #89bdff; 34 | } 35 | /* html/xml tag - light blue */ 36 | pre .atn { 37 | color: #bdb76b; 38 | } 39 | /* html/xml attribute name - khaki */ 40 | pre .atv { 41 | color: #65B042; 42 | } 43 | /* html/xml attribute value - green */ 44 | pre .dec { 45 | color: #3387CC; 46 | } 47 | /* decimal - blue */ 48 | /* Specify class=linenums on a pre to get line numbering */ 49 | ol.linenums { 50 | margin-top: 0; 51 | margin-bottom: 0; 52 | color: #AEAEAE; 53 | } 54 | /* IE indents via margin-left */ 55 | li.L0, 56 | li.L1, 57 | li.L2, 58 | li.L3, 59 | li.L5, 60 | li.L6, 61 | li.L7, 62 | li.L8 { 63 | list-style-type: none; 64 | } 65 | /* Alternate shading for lines */ 66 | @media print { 67 | pre .str { 68 | color: #060; 69 | } 70 | pre .kwd { 71 | color: #006; 72 | font-weight: bold; 73 | } 74 | pre .com { 75 | color: #600; 76 | font-style: italic; 77 | } 78 | pre .typ { 79 | color: #404; 80 | font-weight: bold; 81 | } 82 | pre .lit { 83 | color: #044; 84 | } 85 | pre .pun { 86 | color: #440; 87 | } 88 | pre .pln { 89 | color: #000; 90 | } 91 | pre .tag { 92 | color: #006; 93 | font-weight: bold; 94 | } 95 | pre .atn { 96 | color: #404; 97 | } 98 | pre .atv { 99 | color: #060; 100 | } 101 | } -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-apollo.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["com",/^#[^\n\r]*/,null,"#"],["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^"(?:[^"\\]|\\[\S\s])*(?:"|$)/,null,'"']],[["kwd",/^(?:ADS|AD|AUG|BZF|BZMF|CAE|CAF|CA|CCS|COM|CS|DAS|DCA|DCOM|DCS|DDOUBL|DIM|DOUBLE|DTCB|DTCF|DV|DXCH|EDRUPT|EXTEND|INCR|INDEX|NDX|INHINT|LXCH|MASK|MSK|MP|MSU|NOOP|OVSK|QXCH|RAND|READ|RELINT|RESUME|RETURN|ROR|RXOR|SQUARE|SU|TCR|TCAA|OVSK|TCF|TC|TS|WAND|WOR|WRITE|XCH|XLQ|XXALQ|ZL|ZQ|ADD|ADZ|SUB|SUZ|MPY|MPR|MPZ|DVP|COM|ABS|CLA|CLZ|LDQ|STO|STQ|ALS|LLS|LRS|TRA|TSQ|TMI|TOV|AXT|TIX|DLY|INP|OUT)\s/, 2 | null],["typ",/^(?:-?GENADR|=MINUS|2BCADR|VN|BOF|MM|-?2CADR|-?[1-6]DNADR|ADRES|BBCON|[ES]?BANK=?|BLOCK|BNKSUM|E?CADR|COUNT\*?|2?DEC\*?|-?DNCHAN|-?DNPTR|EQUALS|ERASE|MEMORY|2?OCT|REMADR|SETLOC|SUBRO|ORG|BSS|BES|SYN|EQU|DEFINE|END)\s/,null],["lit",/^'(?:-*(?:\w|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?)?/],["pln",/^-*(?:[!-z]|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?/],["pun",/^[^\w\t\n\r "'-);\\\xa0]+/]]),["apollo","agc","aea"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-basic.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["str",/^"(?:[^\n\r"\\]|\\.)*(?:"|$)/,a,'"'],["pln",/^\s+/,a," \r\n\t\u00a0"]],[["com",/^REM[^\n\r]*/,a],["kwd",/^\b(?:AND|CLOSE|CLR|CMD|CONT|DATA|DEF ?FN|DIM|END|FOR|GET|GOSUB|GOTO|IF|INPUT|LET|LIST|LOAD|NEW|NEXT|NOT|ON|OPEN|OR|POKE|PRINT|READ|RESTORE|RETURN|RUN|SAVE|STEP|STOP|SYS|THEN|TO|VERIFY|WAIT)\b/,a],["pln",/^[a-z][^\W_]?(?:\$|%)?/i,a],["lit",/^(?:\d+(?:\.\d*)?|\.\d+)(?:e[+-]?\d+)?/i,a,"0123456789"],["pun", 3 | /^.[^\s\w"$%.]*/,a]]),["basic","cbm"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-clj.js: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright (C) 2011 Google Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | 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 | var a=null; 17 | PR.registerLangHandler(PR.createSimpleLexer([["opn",/^[([{]+/,a,"([{"],["clo",/^[)\]}]+/,a,")]}"],["com",/^;[^\n\r]*/,a,";"],["pln",/^[\t\n\r \xa0]+/,a,"\t\n\r \u00a0"],["str",/^"(?:[^"\\]|\\[\S\s])*(?:"|$)/,a,'"']],[["kwd",/^(?:def|if|do|let|quote|var|fn|loop|recur|throw|try|monitor-enter|monitor-exit|defmacro|defn|defn-|macroexpand|macroexpand-1|for|doseq|dosync|dotimes|and|or|when|not|assert|doto|proxy|defstruct|first|rest|cons|defprotocol|deftype|defrecord|reify|defmulti|defmethod|meta|with-meta|ns|in-ns|create-ns|import|intern|refer|alias|namespace|resolve|ref|deref|refset|new|set!|memfn|to-array|into-array|aset|gen-class|reduce|map|filter|find|nil?|empty?|hash-map|hash-set|vec|vector|seq|flatten|reverse|assoc|dissoc|list|list?|disj|get|union|difference|intersection|extend|extend-type|extend-protocol|prn)\b/,a], 18 | ["typ",/^:[\dA-Za-z-]+/]]),["clj"]); 19 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-css.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\f\r ]+/,null," \t\r\n\u000c"]],[["str",/^"(?:[^\n\f\r"\\]|\\(?:\r\n?|\n|\f)|\\[\S\s])*"/,null],["str",/^'(?:[^\n\f\r'\\]|\\(?:\r\n?|\n|\f)|\\[\S\s])*'/,null],["lang-css-str",/^url\(([^"')]+)\)/i],["kwd",/^(?:url|rgb|!important|@import|@page|@media|@charset|inherit)(?=[^\w-]|$)/i,null],["lang-css-kw",/^(-?(?:[_a-z]|\\[\da-f]+ ?)(?:[\w-]|\\\\[\da-f]+ ?)*)\s*:/i],["com",/^\/\*[^*]*\*+(?:[^*/][^*]*\*+)*\//], 2 | ["com",/^(?:<\!--|--\>)/],["lit",/^(?:\d+|\d*\.\d+)(?:%|[a-z]+)?/i],["lit",/^#[\da-f]{3,6}\b/i],["pln",/^-?(?:[_a-z]|\\[\da-f]+ ?)(?:[\w-]|\\\\[\da-f]+ ?)*/i],["pun",/^[^\s\w"']+/]]),["css"]);PR.registerLangHandler(PR.createSimpleLexer([],[["kwd",/^-?(?:[_a-z]|\\[\da-f]+ ?)(?:[\w-]|\\\\[\da-f]+ ?)*/i]]),["css-kw"]);PR.registerLangHandler(PR.createSimpleLexer([],[["str",/^[^"')]+/]]),["css-str"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-dart.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"]],[["com",/^#!.*/],["kwd",/^\b(?:import|library|part of|part|as|show|hide)\b/i],["com",/^\/\/.*/],["com",/^\/\*[^*]*\*+(?:[^*/][^*]*\*+)*\//],["kwd",/^\b(?:class|interface)\b/i],["kwd",/^\b(?:assert|break|case|catch|continue|default|do|else|finally|for|if|in|is|new|return|super|switch|this|throw|try|while)\b/i],["kwd",/^\b(?:abstract|const|extends|factory|final|get|implements|native|operator|set|static|typedef|var)\b/i], 2 | ["typ",/^\b(?:bool|double|dynamic|int|num|object|string|void)\b/i],["kwd",/^\b(?:false|null|true)\b/i],["str",/^r?'''[\S\s]*?[^\\]'''/],["str",/^r?"""[\S\s]*?[^\\]"""/],["str",/^r?'('|[^\n\f\r]*?[^\\]')/],["str",/^r?"("|[^\n\f\r]*?[^\\]")/],["pln",/^[$_a-z]\w*/i],["pun",/^[!%&*+/:<-?^|~-]/],["lit",/^\b0x[\da-f]+/i],["lit",/^\b\d+(?:\.\d*)?(?:e[+-]?\d+)?/i],["lit",/^\b\.\d+(?:e[+-]?\d+)?/i],["pun",/^[(),.;[\]{}]/]]), 3 | ["dart"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-erlang.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t-\r ]+/,null,"\t\n\u000b\u000c\r "],["str",/^"(?:[^\n\f\r"\\]|\\[\S\s])*(?:"|$)/,null,'"'],["lit",/^[a-z]\w*/],["lit",/^'(?:[^\n\f\r'\\]|\\[^&])+'?/,null,"'"],["lit",/^\?[^\t\n ({]+/,null,"?"],["lit",/^(?:0o[0-7]+|0x[\da-f]+|\d+(?:\.\d+)?(?:e[+-]?\d+)?)/i,null,"0123456789"]],[["com",/^%[^\n]*/],["kwd",/^(?:module|attributes|do|let|in|letrec|apply|call|primop|case|of|end|when|fun|try|catch|receive|after|char|integer|float,atom,string,var)\b/], 2 | ["kwd",/^-[_a-z]+/],["typ",/^[A-Z_]\w*/],["pun",/^[,.;]/]]),["erlang","erl"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-go.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["pln",/^(?:"(?:[^"\\]|\\[\S\s])*(?:"|$)|'(?:[^'\\]|\\[\S\s])+(?:'|$)|`[^`]*(?:`|$))/,null,"\"'"]],[["com",/^(?:\/\/[^\n\r]*|\/\*[\S\s]*?\*\/)/],["pln",/^(?:[^"'/`]|\/(?![*/]))+/]]),["go"]); 2 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-hs.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t-\r ]+/,null,"\t\n\u000b\u000c\r "],["str",/^"(?:[^\n\f\r"\\]|\\[\S\s])*(?:"|$)/,null,'"'],["str",/^'(?:[^\n\f\r'\\]|\\[^&])'?/,null,"'"],["lit",/^(?:0o[0-7]+|0x[\da-f]+|\d+(?:\.\d+)?(?:e[+-]?\d+)?)/i,null,"0123456789"]],[["com",/^(?:--+[^\n\f\r]*|{-(?:[^-]|-+[^}-])*-})/],["kwd",/^(?:case|class|data|default|deriving|do|else|if|import|in|infix|infixl|infixr|instance|let|module|newtype|of|then|type|where|_)(?=[^\d'A-Za-z]|$)/, 2 | null],["pln",/^(?:[A-Z][\w']*\.)*[A-Za-z][\w']*/],["pun",/^[^\d\t-\r "'A-Za-z]+/]]),["hs"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-lisp.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["opn",/^\(+/,a,"("],["clo",/^\)+/,a,")"],["com",/^;[^\n\r]*/,a,";"],["pln",/^[\t\n\r \xa0]+/,a,"\t\n\r \u00a0"],["str",/^"(?:[^"\\]|\\[\S\s])*(?:"|$)/,a,'"']],[["kwd",/^(?:block|c[ad]+r|catch|con[ds]|def(?:ine|un)|do|eq|eql|equal|equalp|eval-when|flet|format|go|if|labels|lambda|let|load-time-value|locally|macrolet|multiple-value-call|nil|progn|progv|quote|require|return-from|setq|symbol-macrolet|t|tagbody|the|throw|unwind)\b/,a], 3 | ["lit",/^[+-]?(?:[#0]x[\da-f]+|\d+\/\d+|(?:\.\d+|\d+(?:\.\d*)?)(?:[de][+-]?\d+)?)/i],["lit",/^'(?:-*(?:\w|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?)?/],["pln",/^-*(?:[_a-z]|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?/i],["pun",/^[^\w\t\n\r "'-);\\\xa0]+/]]),["cl","el","lisp","lsp","scm","ss","rkt"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-llvm.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^!?"(?:[^"\\]|\\[\S\s])*(?:"|$)/,null,'"'],["com",/^;[^\n\r]*/,null,";"]],[["pln",/^[!%@](?:[$\-.A-Z_a-z][\w$\-.]*|\d+)/],["kwd",/^[^\W\d]\w*/,null],["lit",/^\d+\.\d+/],["lit",/^(?:\d+|0[Xx][\dA-Fa-f]+)/],["pun",/^[(-*,:<->[\]{}]|\.\.\.$/]]),["llvm","ll"]); 2 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-lua.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^(?:"(?:[^"\\]|\\[\S\s])*(?:"|$)|'(?:[^'\\]|\\[\S\s])*(?:'|$))/,null,"\"'"]],[["com",/^--(?:\[(=*)\[[\S\s]*?(?:]\1]|$)|[^\n\r]*)/],["str",/^\[(=*)\[[\S\s]*?(?:]\1]|$)/],["kwd",/^(?:and|break|do|else|elseif|end|false|for|function|if|in|local|nil|not|or|repeat|return|then|true|until|while)\b/,null],["lit",/^[+-]?(?:0x[\da-f]+|(?:\.\d+|\d+(?:\.\d*)?)(?:e[+-]?\d+)?)/i], 2 | ["pln",/^[_a-z]\w*/i],["pun",/^[^\w\t\n\r \xa0][^\w\t\n\r "'+=\xa0-]*/]]),["lua"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-ml.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["com",/^#(?:if[\t\n\r \xa0]+(?:[$_a-z][\w']*|``[^\t\n\r`]*(?:``|$))|else|endif|light)/i,null,"#"],["str",/^(?:"(?:[^"\\]|\\[\S\s])*(?:"|$)|'(?:[^'\\]|\\[\S\s])(?:'|$))/,null,"\"'"]],[["com",/^(?:\/\/[^\n\r]*|\(\*[\S\s]*?\*\))/],["kwd",/^(?:abstract|and|as|assert|begin|class|default|delegate|do|done|downcast|downto|elif|else|end|exception|extern|false|finally|for|fun|function|if|in|inherit|inline|interface|internal|lazy|let|match|member|module|mutable|namespace|new|null|of|open|or|override|private|public|rec|return|static|struct|then|to|true|try|type|upcast|use|val|void|when|while|with|yield|asr|land|lor|lsl|lsr|lxor|mod|sig|atomic|break|checked|component|const|constraint|constructor|continue|eager|event|external|fixed|functor|global|include|method|mixin|object|parallel|process|protected|pure|sealed|trait|virtual|volatile)\b/], 2 | ["lit",/^[+-]?(?:0x[\da-f]+|(?:\.\d+|\d+(?:\.\d*)?)(?:e[+-]?\d+)?)/i],["pln",/^(?:[_a-z][\w']*[!#?]?|``[^\t\n\r`]*(?:``|$))/i],["pun",/^[^\w\t\n\r "'\xa0]+/]]),["fs","ml"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-mumps.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^"(?:[^"]|\\.)*"/,null,'"']],[["com",/^;[^\n\r]*/,null,";"],["dec",/^\$(?:d|device|ec|ecode|es|estack|et|etrap|h|horolog|i|io|j|job|k|key|p|principal|q|quit|st|stack|s|storage|sy|system|t|test|tl|tlevel|tr|trestart|x|y|z[a-z]*|a|ascii|c|char|d|data|e|extract|f|find|fn|fnumber|g|get|j|justify|l|length|na|name|o|order|p|piece|ql|qlength|qs|qsubscript|q|query|r|random|re|reverse|s|select|st|stack|t|text|tr|translate|nan)\b/i, 2 | null],["kwd",/^(?:[^$]b|break|c|close|d|do|e|else|f|for|g|goto|h|halt|h|hang|i|if|j|job|k|kill|l|lock|m|merge|n|new|o|open|q|quit|r|read|s|set|tc|tcommit|tre|trestart|tro|trollback|ts|tstart|u|use|v|view|w|write|x|xecute)\b/i,null],["lit",/^[+-]?(?:\.\d+|\d+(?:\.\d*)?)(?:e[+-]?\d+)?/i],["pln",/^[a-z][^\W_]*/i],["pun",/^[^\w\t\n\r"$%;^\xa0]|_/]]),["mumps"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-n.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["str",/^(?:'(?:[^\n\r'\\]|\\.)*'|"(?:[^\n\r"\\]|\\.)*(?:"|$))/,a,'"'],["com",/^#(?:(?:define|elif|else|endif|error|ifdef|include|ifndef|line|pragma|undef|warning)\b|[^\n\r]*)/,a,"#"],["pln",/^\s+/,a," \r\n\t\u00a0"]],[["str",/^@"(?:[^"]|"")*(?:"|$)/,a],["str",/^<#[^#>]*(?:#>|$)/,a],["str",/^<(?:(?:(?:\.\.\/)*|\/?)(?:[\w-]+(?:\/[\w-]+)+)?[\w-]+\.h|[a-z]\w*)>/,a],["com",/^\/\/[^\n\r]*/,a],["com",/^\/\*[\S\s]*?(?:\*\/|$)/, 3 | a],["kwd",/^(?:abstract|and|as|base|catch|class|def|delegate|enum|event|extern|false|finally|fun|implements|interface|internal|is|macro|match|matches|module|mutable|namespace|new|null|out|override|params|partial|private|protected|public|ref|sealed|static|struct|syntax|this|throw|true|try|type|typeof|using|variant|virtual|volatile|when|where|with|assert|assert2|async|break|checked|continue|do|else|ensures|for|foreach|if|late|lock|new|nolate|otherwise|regexp|repeat|requires|return|surroundwith|unchecked|unless|using|while|yield)\b/, 4 | a],["typ",/^(?:array|bool|byte|char|decimal|double|float|int|list|long|object|sbyte|short|string|ulong|uint|ufloat|ulong|ushort|void)\b/,a],["lit",/^@[$_a-z][\w$@]*/i,a],["typ",/^@[A-Z]+[a-z][\w$@]*/,a],["pln",/^'?[$_a-z][\w$@]*/i,a],["lit",/^(?:0x[\da-f]+|(?:\d(?:_\d+)*\d*(?:\.\d*)?|\.\d\+)(?:e[+-]?\d+)?)[a-z]*/i,a,"0123456789"],["pun",/^.[^\s\w"-$'./@`]*/,a]]),["n","nemerle"]); 5 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-pascal.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["str",/^'(?:[^\n\r'\\]|\\.)*(?:'|$)/,a,"'"],["pln",/^\s+/,a," \r\n\t\u00a0"]],[["com",/^\(\*[\S\s]*?(?:\*\)|$)|^{[\S\s]*?(?:}|$)/,a],["kwd",/^(?:absolute|and|array|asm|assembler|begin|case|const|constructor|destructor|div|do|downto|else|end|external|for|forward|function|goto|if|implementation|in|inline|interface|interrupt|label|mod|not|object|of|or|packed|procedure|program|record|repeat|set|shl|shr|then|to|type|unit|until|uses|var|virtual|while|with|xor)\b/i,a], 3 | ["lit",/^(?:true|false|self|nil)/i,a],["pln",/^[a-z][^\W_]*/i,a],["lit",/^(?:\$[\da-f]+|(?:\d+(?:\.\d*)?|\.\d+)(?:e[+-]?\d+)?)/i,a,"0123456789"],["pun",/^.[^\s\w$'./@]*/,a]]),["pascal"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-proto.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.sourceDecorator({keywords:"bytes,default,double,enum,extend,extensions,false,group,import,max,message,option,optional,package,repeated,required,returns,rpc,service,syntax,to,true",types:/^(bool|(double|s?fixed|[su]?int)(32|64)|float|string)\b/,cStyleComments:!0}),["proto"]); 2 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-r.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^"(?:[^"\\]|\\[\S\s])*(?:"|$)/,null,'"'],["str",/^'(?:[^'\\]|\\[\S\s])*(?:'|$)/,null,"'"]],[["com",/^#.*/],["kwd",/^(?:if|else|for|while|repeat|in|next|break|return|switch|function)(?![\w.])/],["lit",/^0[Xx][\dA-Fa-f]+([Pp]\d+)?[Li]?/],["lit",/^[+-]?(\d+(\.\d+)?|\.\d+)([Ee][+-]?\d+)?[Li]?/],["lit",/^(?:NULL|NA(?:_(?:integer|real|complex|character)_)?|Inf|TRUE|FALSE|NaN|\.\.(?:\.|\d+))(?![\w.])/], 2 | ["pun",/^(?:<>?|-|==|<=|>=|<|>|&&?|!=|\|\|?|[!*+/^]|%.*?%|[$=@~]|:{1,3}|[(),;?[\]{}])/],["pln",/^(?:[A-Za-z]+[\w.]*|\.[^\W\d][\w.]*)(?![\w.])/],["str",/^`.+`/]]),["r","s","R","S","Splus"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-rd.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["com",/^%[^\n\r]*/,null,"%"]],[["lit",/^\\(?:cr|l?dots|R|tab)\b/],["kwd",/^\\[@-Za-z]+/],["kwd",/^#(?:ifn?def|endif)/],["pln",/^\\[{}]/],["pun",/^[()[\]{}]+/]]),["Rd","rd"]); 2 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-scala.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^"(?:""(?:""?(?!")|[^"\\]|\\.)*"{0,3}|(?:[^\n\r"\\]|\\.)*"?)/,null,'"'],["lit",/^`(?:[^\n\r\\`]|\\.)*`?/,null,"`"],["pun",/^[!#%&(--:-@[-^{-~]+/,null,"!#%&()*+,-:;<=>?@[\\]^{|}~"]],[["str",/^'(?:[^\n\r'\\]|\\(?:'|[^\n\r']+))'/],["lit",/^'[$A-Z_a-z][\w$]*(?![\w$'])/],["kwd",/^(?:abstract|case|catch|class|def|do|else|extends|final|finally|for|forSome|if|implicit|import|lazy|match|new|object|override|package|private|protected|requires|return|sealed|super|throw|trait|try|type|val|var|while|with|yield)\b/], 2 | ["lit",/^(?:true|false|null|this)\b/],["lit",/^(?:0(?:[0-7]+|x[\da-f]+)l?|(?:0|[1-9]\d*)(?:(?:\.\d+)?(?:e[+-]?\d+)?f?|l?)|\\.\d+(?:e[+-]?\d+)?f?)/i],["typ",/^[$_]*[A-Z][\d$A-Z_]*[a-z][\w$]*/],["pln",/^[$A-Z_a-z][\w$]*/],["com",/^\/(?:\/.*|\*(?:\/|\**[^*/])*(?:\*+\/?)?)/],["pun",/^(?:\.+|\/)/]]),["scala"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-sql.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["str",/^(?:"(?:[^"\\]|\\.)*"|'(?:[^'\\]|\\.)*')/,null,"\"'"]],[["com",/^(?:--[^\n\r]*|\/\*[\S\s]*?(?:\*\/|$))/],["kwd",/^(?:add|all|alter|and|any|apply|as|asc|authorization|backup|begin|between|break|browse|bulk|by|cascade|case|check|checkpoint|close|clustered|coalesce|collate|column|commit|compute|connect|constraint|contains|containstable|continue|convert|create|cross|current|current_date|current_time|current_timestamp|current_user|cursor|database|dbcc|deallocate|declare|default|delete|deny|desc|disk|distinct|distributed|double|drop|dummy|dump|else|end|errlvl|escape|except|exec|execute|exists|exit|fetch|file|fillfactor|following|for|foreign|freetext|freetexttable|from|full|function|goto|grant|group|having|holdlock|identity|identitycol|identity_insert|if|in|index|inner|insert|intersect|into|is|join|key|kill|left|like|lineno|load|match|matched|merge|natural|national|nocheck|nonclustered|nocycle|not|null|nullif|of|off|offsets|on|open|opendatasource|openquery|openrowset|openxml|option|or|order|outer|over|partition|percent|pivot|plan|preceding|precision|primary|print|proc|procedure|public|raiserror|read|readtext|reconfigure|references|replication|restore|restrict|return|revoke|right|rollback|rowcount|rowguidcol|rows?|rule|save|schema|select|session_user|set|setuser|shutdown|some|start|statistics|system_user|table|textsize|then|to|top|tran|transaction|trigger|truncate|tsequal|unbounded|union|unique|unpivot|update|updatetext|use|user|using|values|varying|view|waitfor|when|where|while|with|within|writetext|xml)(?=[^\w-]|$)/i, 2 | null],["lit",/^[+-]?(?:0x[\da-f]+|(?:\.\d+|\d+(?:\.\d*)?)(?:e[+-]?\d+)?)/i],["pln",/^[_a-z][\w-]*/i],["pun",/^[^\w\t\n\r "'\xa0][^\w\t\n\r "'+\xa0-]*/]]),["sql"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-tcl.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["opn",/^{+/,a,"{"],["clo",/^}+/,a,"}"],["com",/^#[^\n\r]*/,a,"#"],["pln",/^[\t\n\r \xa0]+/,a,"\t\n\r \u00a0"],["str",/^"(?:[^"\\]|\\[\S\s])*(?:"|$)/,a,'"']],[["kwd",/^(?:after|append|apply|array|break|case|catch|continue|error|eval|exec|exit|expr|for|foreach|if|incr|info|proc|return|set|switch|trace|uplevel|upvar|while)\b/,a],["lit",/^[+-]?(?:[#0]x[\da-f]+|\d+\/\d+|(?:\.\d+|\d+(?:\.\d*)?)(?:[de][+-]?\d+)?)/i],["lit", 3 | /^'(?:-*(?:\w|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?)?/],["pln",/^-*(?:[_a-z]|\\[!-~])(?:[\w-]*|\\[!-~])[!=?]?/i],["pun",/^[^\w\t\n\r "'-);\\\xa0]+/]]),["tcl"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-tex.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"],["com",/^%[^\n\r]*/,null,"%"]],[["kwd",/^\\[@-Za-z]+/],["kwd",/^\\./],["typ",/^[$&]/],["lit",/[+-]?(?:\.\d+|\d+(?:\.\d*)?)(cm|em|ex|in|pc|pt|bp|mm)/i],["pun",/^[()=[\]{}]+/]]),["latex","tex"]); 2 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-vb.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0\u2028\u2029]+/,null,"\t\n\r \u00a0\u2028\u2029"],["str",/^(?:["\u201c\u201d](?:[^"\u201c\u201d]|["\u201c\u201d]{2})(?:["\u201c\u201d]c|$)|["\u201c\u201d](?:[^"\u201c\u201d]|["\u201c\u201d]{2})*(?:["\u201c\u201d]|$))/i,null,'"\u201c\u201d'],["com",/^['\u2018\u2019](?:_(?:\r\n?|[^\r]?)|[^\n\r_\u2028\u2029])*/,null,"'\u2018\u2019"]],[["kwd",/^(?:addhandler|addressof|alias|and|andalso|ansi|as|assembly|auto|boolean|byref|byte|byval|call|case|catch|cbool|cbyte|cchar|cdate|cdbl|cdec|char|cint|class|clng|cobj|const|cshort|csng|cstr|ctype|date|decimal|declare|default|delegate|dim|directcast|do|double|each|else|elseif|end|endif|enum|erase|error|event|exit|finally|for|friend|function|get|gettype|gosub|goto|handles|if|implements|imports|in|inherits|integer|interface|is|let|lib|like|long|loop|me|mod|module|mustinherit|mustoverride|mybase|myclass|namespace|new|next|not|notinheritable|notoverridable|object|on|option|optional|or|orelse|overloads|overridable|overrides|paramarray|preserve|private|property|protected|public|raiseevent|readonly|redim|removehandler|resume|return|select|set|shadows|shared|short|single|static|step|stop|string|structure|sub|synclock|then|throw|to|try|typeof|unicode|until|variant|wend|when|while|with|withevents|writeonly|xor|endif|gosub|let|variant|wend)\b/i, 2 | null],["com",/^rem\b.*/i],["lit",/^(?:true\b|false\b|nothing\b|\d+(?:e[+-]?\d+[dfr]?|[dfilrs])?|(?:&h[\da-f]+|&o[0-7]+)[ils]?|\d*\.\d+(?:e[+-]?\d+)?[dfr]?|#\s+(?:\d+[/-]\d+[/-]\d+(?:\s+\d+:\d+(?::\d+)?(\s*(?:am|pm))?)?|\d+:\d+(?::\d+)?(\s*(?:am|pm))?)\s+#)/i],["pln",/^(?:(?:[a-z]|_\w)\w*(?:\[[!#%&@]+])?|\[(?:[a-z]|_\w)\w*])/i],["pun",/^[^\w\t\n\r "'[\]\xa0\u2018\u2019\u201c\u201d\u2028\u2029]+/],["pun",/^(?:\[|])/]]),["vb","vbs"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-vhdl.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\t\n\r \xa0]+/,null,"\t\n\r \u00a0"]],[["str",/^(?:[box]?"(?:[^"]|"")*"|'.')/i],["com",/^--[^\n\r]*/],["kwd",/^(?:abs|access|after|alias|all|and|architecture|array|assert|attribute|begin|block|body|buffer|bus|case|component|configuration|constant|disconnect|downto|else|elsif|end|entity|exit|file|for|function|generate|generic|group|guarded|if|impure|in|inertial|inout|is|label|library|linkage|literal|loop|map|mod|nand|new|next|nor|not|null|of|on|open|or|others|out|package|port|postponed|procedure|process|pure|range|record|register|reject|rem|report|return|rol|ror|select|severity|shared|signal|sla|sll|sra|srl|subtype|then|to|transport|type|unaffected|units|until|use|variable|wait|when|while|with|xnor|xor)(?=[^\w-]|$)/i, 2 | null],["typ",/^(?:bit|bit_vector|character|boolean|integer|real|time|string|severity_level|positive|natural|signed|unsigned|line|text|std_u?logic(?:_vector)?)(?=[^\w-]|$)/i,null],["typ",/^'(?:active|ascending|base|delayed|driving|driving_value|event|high|image|instance_name|last_active|last_event|last_value|left|leftof|length|low|path_name|pos|pred|quiet|range|reverse_range|right|rightof|simple_name|stable|succ|transaction|val|value)(?=[^\w-]|$)/i,null],["lit",/^\d+(?:_\d+)*(?:#[\w.\\]+#(?:[+-]?\d+(?:_\d+)*)?|(?:\.\d+(?:_\d+)*)?(?:e[+-]?\d+(?:_\d+)*)?)/i], 3 | ["pln",/^(?:[a-z]\w*|\\[^\\]*\\)/i],["pun",/^[^\w\t\n\r "'\xa0][^\w\t\n\r "'\xa0-]*/]]),["vhdl","vhd"]); 4 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-wiki.js: -------------------------------------------------------------------------------- 1 | PR.registerLangHandler(PR.createSimpleLexer([["pln",/^[\d\t a-gi-z\xa0]+/,null,"\t \u00a0abcdefgijklmnopqrstuvwxyz0123456789"],["pun",/^[*=[\]^~]+/,null,"=*~^[]"]],[["lang-wiki.meta",/(?:^^|\r\n?|\n)(#[a-z]+)\b/],["lit",/^[A-Z][a-z][\da-z]+[A-Z][a-z][^\W_]+\b/],["lang-",/^{{{([\S\s]+?)}}}/],["lang-",/^`([^\n\r`]+)`/],["str",/^https?:\/\/[^\s#/?]*(?:\/[^\s#?]*)?(?:\?[^\s#]*)?(?:#\S*)?/i],["pln",/^(?:\r\n|[\S\s])[^\n\r#*=A-[^`h{~]*/]]),["wiki"]); 2 | PR.registerLangHandler(PR.createSimpleLexer([["kwd",/^#[a-z]+/i,null,"#"]],[]),["wiki.meta"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/lang-yaml.js: -------------------------------------------------------------------------------- 1 | var a=null; 2 | PR.registerLangHandler(PR.createSimpleLexer([["pun",/^[:>?|]+/,a,":|>?"],["dec",/^%(?:YAML|TAG)[^\n\r#]+/,a,"%"],["typ",/^&\S+/,a,"&"],["typ",/^!\S*/,a,"!"],["str",/^"(?:[^"\\]|\\.)*(?:"|$)/,a,'"'],["str",/^'(?:[^']|'')*(?:'|$)/,a,"'"],["com",/^#[^\n\r]*/,a,"#"],["pln",/^\s+/,a," \t\r\n"]],[["dec",/^(?:---|\.\.\.)(?:[\n\r]|$)/],["pun",/^-/],["kwd",/^\w+:[\n\r ]/],["pln",/^\w+/]]),["yaml","yml"]); 3 | -------------------------------------------------------------------------------- /src/main/resources/apidoc/vendor/prettify/prettify.css: -------------------------------------------------------------------------------- 1 | .pln{color:#000}@media screen{.str{color:#080}.kwd{color:#008}.com{color:#800}.typ{color:#606}.lit{color:#066}.pun,.opn,.clo{color:#660}.tag{color:#008}.atn{color:#606}.atv{color:#080}.dec,.var{color:#606}.fun{color:red}}@media print,projection{.str{color:#060}.kwd{color:#006;font-weight:bold}.com{color:#600;font-style:italic}.typ{color:#404;font-weight:bold}.lit{color:#044}.pun,.opn,.clo{color:#440}.tag{color:#006;font-weight:bold}.atn{color:#404}.atv{color:#060}}pre.prettyprint{padding:2px;border:1px solid #888}ol.linenums{margin-top:0;margin-bottom:0}li.L0,li.L1,li.L2,li.L3,li.L5,li.L6,li.L7,li.L8{list-style-type:none}li.L1,li.L3,li.L5,li.L7,li.L9{background:#eee} -------------------------------------------------------------------------------- /src/main/resources/dfa/asset-manifest.json: -------------------------------------------------------------------------------- 1 | { 2 | "main.css": "static/css/main.5631a031.css", 3 | "main.css.map": "static/css/main.5631a031.css.map", 4 | "main.js": "static/js/main.a334a047.js", 5 | "main.js.map": "static/js/main.a334a047.js.map" 6 | } -------------------------------------------------------------------------------- /src/main/resources/dfa/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/dfa/favicon.ico -------------------------------------------------------------------------------- /src/main/resources/dfa/index.html: -------------------------------------------------------------------------------- 1 | DataFibers
Loading...
-------------------------------------------------------------------------------- /src/main/resources/dfa/service-worker.js: -------------------------------------------------------------------------------- 1 | "use strict";function setOfCachedUrls(e){return e.keys().then(function(e){return e.map(function(e){return e.url})}).then(function(e){return new Set(e)})}var precacheConfig=[["index.html","fe848b99b936e2fa8fbadf941fa70f55"],["static/css/main.5631a031.css","14994677c8b5e679c7dc9a384a7092ba"],["static/js/main.a334a047.js","aa0d6a5b8ed736a3ba8d4b8df3d571cc"]],cacheName="sw-precache-v3-sw-precache-webpack-plugin-"+(self.registration?self.registration.scope:""),ignoreUrlParametersMatching=[/^utm_/],addDirectoryIndex=function(e,t){var n=new URL(e);return"/"===n.pathname.slice(-1)&&(n.pathname+=t),n.toString()},cleanResponse=function(e){return e.redirected?("body"in e?Promise.resolve(e.body):e.blob()).then(function(t){return new Response(t,{headers:e.headers,status:e.status,statusText:e.statusText})}):Promise.resolve(e)},createCacheKey=function(e,t,n,r){var a=new URL(e);return r&&a.pathname.match(r)||(a.search+=(a.search?"&":"")+encodeURIComponent(t)+"="+encodeURIComponent(n)),a.toString()},isPathWhitelisted=function(e,t){if(0===e.length)return!0;var n=new URL(t).pathname;return e.some(function(e){return n.match(e)})},stripIgnoredUrlParameters=function(e,t){var n=new URL(e);return n.hash="",n.search=n.search.slice(1).split("&").map(function(e){return e.split("=")}).filter(function(e){return t.every(function(t){return!t.test(e[0])})}).map(function(e){return e.join("=")}).join("&"),n.toString()},hashParamName="_sw-precache",urlsToCacheKeys=new Map(precacheConfig.map(function(e){var t=e[0],n=e[1],r=new URL(t,self.location),a=createCacheKey(r,hashParamName,n,/\.\w{8}\./);return[r.toString(),a]}));self.addEventListener("install",function(e){e.waitUntil(caches.open(cacheName).then(function(e){return setOfCachedUrls(e).then(function(t){return Promise.all(Array.from(urlsToCacheKeys.values()).map(function(n){if(!t.has(n)){var r=new Request(n,{credentials:"same-origin"});return fetch(r).then(function(t){if(!t.ok)throw new Error("Request for "+n+" returned a response with status "+t.status);return cleanResponse(t).then(function(t){return e.put(n,t)})})}}))})}).then(function(){return self.skipWaiting()}))}),self.addEventListener("activate",function(e){var t=new Set(urlsToCacheKeys.values());e.waitUntil(caches.open(cacheName).then(function(e){return e.keys().then(function(n){return Promise.all(n.map(function(n){if(!t.has(n.url))return e.delete(n)}))})}).then(function(){return self.clients.claim()}))}),self.addEventListener("fetch",function(e){if("GET"===e.request.method){var t,n=stripIgnoredUrlParameters(e.request.url,ignoreUrlParametersMatching);t=urlsToCacheKeys.has(n);t||(n=addDirectoryIndex(n,"index.html"),t=urlsToCacheKeys.has(n));!t&&"navigate"===e.request.mode&&isPathWhitelisted([],e.request.url)&&(n=new URL("/dfa/index.html",self.location).toString(),t=urlsToCacheKeys.has(n)),t&&e.respondWith(caches.open(cacheName).then(function(e){return e.match(urlsToCacheKeys.get(n)).then(function(e){if(e)return e;throw Error("The cached response that was expected is missing.")})}).catch(function(t){return console.warn('Couldn\'t serve response for "%s" from cache: %O',e.request.url,t),fetch(e.request)}))}}); -------------------------------------------------------------------------------- /src/main/resources/import/df_installed.json: -------------------------------------------------------------------------------- 1 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_KAFKA_JDBC","class":"io.confluent.connect.jdbc.JdbcSourceConnector","name":"JDBC Source Connector","type":"Kafka","subtype":"Source","certified":"datafibers","pubisher":"confluent","link":"http://www.confluent.io","version":"3.3.0","config":null} 2 | {"meta_type":"installed_connect","connectorType":"CONNECT_SINK_KAFKA_JDBC","class":"io.confluent.connect.jdbc.JdbcSinkConnector","name":"JDBC Sink Connector","type":"Kafka","subtype":"Sink","certified":"datafibers","pubisher":"confluent","link":"http://www.confluent.io","version":"3.3.0","config":null} 3 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_KAFKA_AvroFile","class":"com.datafibers.kafka.connect.FileGenericSourceConnector","name":"FileGeneric Source Connector","type":"Kafka","subtype":"Source","certified":"datafibers","pubisher":"datafibers","link":"https://github.com/datafibers-community/df_certified_connects","version":"0.1.0","config":null} 4 | {"meta_type":"installed_connect","connectorType":"CONNECT_SINK_KAFKA_SchemaFile","class":"com.datafibers.kafka.connect.SchemaedFileSinkConnector","name":"SchemaedFile Sink Connector","type":"Kafka","subtype":"Sink","certified":"to be certified","pubisher":"dbtucker","link":"https://github.com/dbtucker/kafka-connect-schemaedfile","version":"0.1.0","config":null} 5 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_KAFKA_SchemaFile","class":"com.datafibers.kafka.connect.SchemaedFileSourceConnector","name":"SchemaedFile Source Connector","type":"Kafka","subtype":"Source","certified":"to be certified","pubisher":"dbtucker","link":"https://github.com/dbtucker/kafka-connect-schemaedfile","version":"0.1.0","config":null} 6 | {"meta_type":"installed_connect","connectorType":"CONNECT_SINK_HDFS_AvroFile","class":"io.confluent.connect.hdfs.HdfsSinkConnector","name":"HDFS Sink Connector","type":"Kafka","subtype":"Sink","certified":"to be certified","pubisher":"confluent","link":"http://www.confluent.io","version":"0.1.0","config":null} 7 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_HDFS_AvroFile","class":"io.confluent.connect.hdfs.tools.SchemaSourceConnector","name":"HDFS Source Connector","type":"Kafka","subtype":"Source","certified":"to be certified","pubisher":"confluent","link":"http://www.confluent.io","version":"0.1.0","config":null} 8 | {"meta_type":"installed_connect","connectorType":"CONNECT_SINK_FlatFile","class":"org.apache.kafka.connect.file.FileStreamSinkConnector","name":"FileStream Sink Connector","type":"Kafka","subtype":"Sink","certified":"to be certified","pubisher":"confluent","link":"http://www.confluent.io","version":"0.1.0","config":null} 9 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_FlatFile","class":"org.apache.kafka.connect.file.FileStreamSourceConnector","name":"FileStream Source Connector","type":"Kafka","subtype":"Source","certified":"to be certified","pubisher":"confluent","link":"http://www.confluent.io","version":"0.1.0","config":null} 10 | {"meta_type":"installed_connect","connectorType":"CONNECT_SINK_MONGODB_AvroDB","class":"org.apache.kafka.connect.mongodb.MongodbSinkConnector","name":"Mongodb Sink Connector","type":"Kafka","subtype":"Sink","certified":"datafibers","pubisher":"datareply","link":"https://github.com/DataReply/kafka-connect-mongodb","version":"0.1.0","config":null} 11 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_MONGODB_AvroDB","class":"org.apache.kafka.connect.mongodb.MongodbSourceConnector","name":"Mongodb Source Connector","type":"Kafka","subtype":"Source","certified":"datafibers","pubisher":"datareply","link":"https://github.com/DataReply/kafka-connect-mongodb","version":"0.1.0","config":null} 12 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_STOCK_AvroFile","class":"com.datafibers.kafka.connect.FinanceSourceConnector","name":"Finance Stock Connector","type":"Kafka","subtype":"Source","certified":"datafibers","pubisher":"datafibers","link":"http://www.datafibers.com","version":"0.1.0","config":null} 13 | {"meta_type":"installed_connect","connectorType":"CONNECT_SOURCE_UKMEET_NetCDFFile","class":"com.datafibers.kafka.connect.UKMEETSourceConnector","name":"UK MEET Office Connector","type":"Kafka","subtype":"Source","certified":"datafibers","pubisher":"datafibers","link":"http://www.datafibers.com","version":"0.1.0","config":null} -------------------------------------------------------------------------------- /src/main/resources/landing/README.txt: -------------------------------------------------------------------------------- 1 | Eventually by HTML5 UP 2 | html5up.net | @ajlkn 3 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 4 | 5 | 6 | Just a simple placeholder template for your kickass app/product/startup/whatever until it 7 | launches. Includes an email signup form and a cool slideshow background (more on both below). 8 | 9 | Demo images* courtesy of Unsplash, a radtastic collection of CC0 (public domain) images 10 | you can use for pretty much whatever. 11 | 12 | (* = not included) 13 | 14 | AJ 15 | aj@lkn.io | @ajlkn 16 | 17 | 18 | Signup Form: 19 | 20 | The signup form won't actually do anything (other than report back with a "thank you" message) 21 | until you tie it to either a third party service (eg. MailChimp) or your own hosted solution. 22 | In either case, there are two ways to go: 23 | 24 | 1. The conventional (non-AJAX) way, which pretty much comes down to pointing the form's "action" 25 | attribute to your service/script URL. If you go this route, remove the entire "Signup Form" code 26 | block from assets/js/main.js (since it's not needed for this approach). 27 | 28 | -or- 29 | 30 | 2. The AJAX way. How you set this up is largely dependent on the service/solution you're using 31 | so you'll need to consult their/its documentation. However, I have included some basic code 32 | (under "Signup Form" in assets/js/main.js) that will at least let you interact with the 33 | form itself. 34 | 35 | 36 | Slideshow Background: 37 | 38 | This is pretty straightforward, but there are two JS settings you'll want to be aware of 39 | (found under "Slideshow Background" in assets/js/main.js): 40 | 41 | images 42 | 43 | The list of images to cycle through, given in the following format: 44 | 45 | 'url': 'alignment' 46 | 47 | Where 'url' is the image (eg. 'images/foo.jpg', 'http://somewhere.else/foo.jpg'), and 48 | 'alignment' is how the image should be vertically aligned ('top', 'center', or 'bottom'). 49 | 50 | Note: Browsers that don't support CSS transitions (like IE<=9) will only see the first image. 51 | 52 | delay 53 | 54 | How long to wait between transitions (in ms). Note that this must be at least twice as long as 55 | the transition speed itself (currently 3 seconds). 56 | 57 | 58 | Credits: 59 | 60 | Demo Images: 61 | Unsplash (unsplash.com) 62 | 63 | Icons: 64 | Font Awesome (fortawesome.github.com/Font-Awesome) 65 | 66 | Other: 67 | html5shiv.js (@afarkas @jdalton @jon_neal @rem) 68 | Respond.js (j.mp/respondjs) 69 | Skel (skel.io) -------------------------------------------------------------------------------- /src/main/resources/landing/assets/css/ie8.css: -------------------------------------------------------------------------------- 1 | /* 2 | Eventually by HTML5 UP 3 | html5up.net | @ajlkn 4 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 5 | */ 6 | 7 | /* BG */ 8 | 9 | #bg { 10 | -ms-filter: "progid:DXImageTransform.Microsoft.Alpha(Opacity=25)"; 11 | } 12 | 13 | /* Type */ 14 | 15 | body, input, select, textarea { 16 | color: #fff; 17 | } 18 | 19 | /* Form */ 20 | 21 | input[type="text"], 22 | input[type="password"], 23 | input[type="email"], 24 | select, 25 | textarea { 26 | border: solid 2px #fff; 27 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/css/ie9.css: -------------------------------------------------------------------------------- 1 | /* 2 | Eventually by HTML5 UP 3 | html5up.net | @ajlkn 4 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 5 | */ 6 | 7 | /* Signup Form */ 8 | 9 | #signup-form > * { 10 | display: inline-block; 11 | vertical-align: top; 12 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/fonts/FontAwesome.otf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/assets/fonts/FontAwesome.otf -------------------------------------------------------------------------------- /src/main/resources/landing/assets/fonts/fontawesome-webfont.eot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/assets/fonts/fontawesome-webfont.eot -------------------------------------------------------------------------------- /src/main/resources/landing/assets/fonts/fontawesome-webfont.ttf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/assets/fonts/fontawesome-webfont.ttf -------------------------------------------------------------------------------- /src/main/resources/landing/assets/fonts/fontawesome-webfont.woff: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/assets/fonts/fontawesome-webfont.woff -------------------------------------------------------------------------------- /src/main/resources/landing/assets/fonts/fontawesome-webfont.woff2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/assets/fonts/fontawesome-webfont.woff2 -------------------------------------------------------------------------------- /src/main/resources/landing/assets/js/ie/html5shiv.js: -------------------------------------------------------------------------------- 1 | /* 2 | HTML5 Shiv v3.6.2 | @afarkas @jdalton @jon_neal @rem | MIT/GPL2 Licensed 3 | */ 4 | (function(l,f){function m(){var a=e.elements;return"string"==typeof a?a.split(" "):a}function i(a){var b=n[a[o]];b||(b={},h++,a[o]=h,n[h]=b);return b}function p(a,b,c){b||(b=f);if(g)return b.createElement(a);c||(c=i(b));b=c.cache[a]?c.cache[a].cloneNode():r.test(a)?(c.cache[a]=c.createElem(a)).cloneNode():c.createElem(a);return b.canHaveChildren&&!s.test(a)?c.frag.appendChild(b):b}function t(a,b){if(!b.cache)b.cache={},b.createElem=a.createElement,b.createFrag=a.createDocumentFragment,b.frag=b.createFrag(); 5 | a.createElement=function(c){return!e.shivMethods?b.createElem(c):p(c,a,b)};a.createDocumentFragment=Function("h,f","return function(){var n=f.cloneNode(),c=n.createElement;h.shivMethods&&("+m().join().replace(/\w+/g,function(a){b.createElem(a);b.frag.createElement(a);return'c("'+a+'")'})+");return n}")(e,b.frag)}function q(a){a||(a=f);var b=i(a);if(e.shivCSS&&!j&&!b.hasCSS){var c,d=a;c=d.createElement("p");d=d.getElementsByTagName("head")[0]||d.documentElement;c.innerHTML="x"; 6 | c=d.insertBefore(c.lastChild,d.firstChild);b.hasCSS=!!c}g||t(a,b);return a}var k=l.html5||{},s=/^<|^(?:button|map|select|textarea|object|iframe|option|optgroup)$/i,r=/^(?:a|b|code|div|fieldset|h1|h2|h3|h4|h5|h6|i|label|li|ol|p|q|span|strong|style|table|tbody|td|th|tr|ul)$/i,j,o="_html5shiv",h=0,n={},g;(function(){try{var a=f.createElement("a");a.innerHTML="";j="hidden"in a;var b;if(!(b=1==a.childNodes.length)){f.createElement("a");var c=f.createDocumentFragment();b="undefined"==typeof c.cloneNode|| 7 | "undefined"==typeof c.createDocumentFragment||"undefined"==typeof c.createElement}g=b}catch(d){g=j=!0}})();var e={elements:k.elements||"abbr article aside audio bdi canvas data datalist details figcaption figure footer header hgroup main mark meter nav output progress section summary time video",version:"3.6.2",shivCSS:!1!==k.shivCSS,supportsUnknownElements:g,shivMethods:!1!==k.shivMethods,type:"default",shivDocument:q,createElement:p,createDocumentFragment:function(a,b){a||(a=f);if(g)return a.createDocumentFragment(); 8 | for(var b=b||i(a),c=b.frag.cloneNode(),d=0,e=m(),h=e.length;d #mq-test-1 { width: 42px; }',c.insertBefore(e,d),b=42===f.offsetWidth,c.removeChild(e),{matches:b,media:a}}}(a.document)}(this),function(a){"use strict";function b(){v(!0)}var c={};a.respond=c,c.update=function(){};var d=[],e=function(){var b=!1;try{b=new a.XMLHttpRequest}catch(c){b=new a.ActiveXObject("Microsoft.XMLHTTP")}return function(){return b}}(),f=function(a,b){var c=e();c&&(c.open("GET",a,!0),c.onreadystatechange=function(){4!==c.readyState||200!==c.status&&304!==c.status||b(c.responseText)},4!==c.readyState&&c.send(null))},g=function(a){return a.replace(c.regex.minmaxwh,"").match(c.regex.other)};if(c.ajax=f,c.queue=d,c.unsupportedmq=g,c.regex={media:/@media[^\{]+\{([^\{\}]*\{[^\}\{]*\})+/gi,keyframes:/@(?:\-(?:o|moz|webkit)\-)?keyframes[^\{]+\{(?:[^\{\}]*\{[^\}\{]*\})+[^\}]*\}/gi,comments:/\/\*[^*]*\*+([^/][^*]*\*+)*\//gi,urls:/(url\()['"]?([^\/\)'"][^:\)'"]+)['"]?(\))/g,findStyles:/@media *([^\{]+)\{([\S\s]+?)$/,only:/(only\s+)?([a-zA-Z]+)\s?/,minw:/\(\s*min\-width\s*:\s*(\s*[0-9\.]+)(px|em)\s*\)/,maxw:/\(\s*max\-width\s*:\s*(\s*[0-9\.]+)(px|em)\s*\)/,minmaxwh:/\(\s*m(in|ax)\-(height|width)\s*:\s*(\s*[0-9\.]+)(px|em)\s*\)/gi,other:/\([^\)]*\)/g},c.mediaQueriesSupported=a.matchMedia&&null!==a.matchMedia("only all")&&a.matchMedia("only all").matches,!c.mediaQueriesSupported){var h,i,j,k=a.document,l=k.documentElement,m=[],n=[],o=[],p={},q=30,r=k.getElementsByTagName("head")[0]||l,s=k.getElementsByTagName("base")[0],t=r.getElementsByTagName("link"),u=function(){var a,b=k.createElement("div"),c=k.body,d=l.style.fontSize,e=c&&c.style.fontSize,f=!1;return b.style.cssText="position:absolute;font-size:1em;width:1em",c||(c=f=k.createElement("body"),c.style.background="none"),l.style.fontSize="100%",c.style.fontSize="100%",c.appendChild(b),f&&l.insertBefore(c,l.firstChild),a=b.offsetWidth,f?l.removeChild(c):c.removeChild(b),l.style.fontSize=d,e&&(c.style.fontSize=e),a=j=parseFloat(a)},v=function(b){var c="clientWidth",d=l[c],e="CSS1Compat"===k.compatMode&&d||k.body[c]||d,f={},g=t[t.length-1],p=(new Date).getTime();if(b&&h&&q>p-h)return a.clearTimeout(i),i=a.setTimeout(v,q),void 0;h=p;for(var s in m)if(m.hasOwnProperty(s)){var w=m[s],x=w.minw,y=w.maxw,z=null===x,A=null===y,B="em";x&&(x=parseFloat(x)*(x.indexOf(B)>-1?j||u():1)),y&&(y=parseFloat(y)*(y.indexOf(B)>-1?j||u():1)),w.hasquery&&(z&&A||!(z||e>=x)||!(A||y>=e))||(f[w.media]||(f[w.media]=[]),f[w.media].push(n[w.rules]))}for(var C in o)o.hasOwnProperty(C)&&o[C]&&o[C].parentNode===r&&r.removeChild(o[C]);o.length=0;for(var D in f)if(f.hasOwnProperty(D)){var E=k.createElement("style"),F=f[D].join("\n");E.type="text/css",E.media=D,r.insertBefore(E,g.nextSibling),E.styleSheet?E.styleSheet.cssText=F:E.appendChild(k.createTextNode(F)),o.push(E)}},w=function(a,b,d){var e=a.replace(c.regex.comments,"").replace(c.regex.keyframes,"").match(c.regex.media),f=e&&e.length||0;b=b.substring(0,b.lastIndexOf("/"));var h=function(a){return a.replace(c.regex.urls,"$1"+b+"$2$3")},i=!f&&d;b.length&&(b+="/"),i&&(f=1);for(var j=0;f>j;j++){var k,l,o,p;i?(k=d,n.push(h(a))):(k=e[j].match(c.regex.findStyles)&&RegExp.$1,n.push(RegExp.$2&&h(RegExp.$2))),o=k.split(","),p=o.length;for(var q=0;p>q;q++)l=o[q],g(l)||m.push({media:l.split("(")[0].match(c.regex.only)&&RegExp.$2||"all",rules:n.length-1,hasquery:l.indexOf("(")>-1,minw:l.match(c.regex.minw)&&parseFloat(RegExp.$1)+(RegExp.$2||""),maxw:l.match(c.regex.maxw)&&parseFloat(RegExp.$1)+(RegExp.$2||"")})}v()},x=function(){if(d.length){var b=d.shift();f(b.href,function(c){w(c,b.href,b.media),p[b.href]=!0,a.setTimeout(function(){x()},0)})}},y=function(){for(var b=0;b= $bgs.length) 94 | pos = 0; 95 | 96 | // Swap top images. 97 | $bgs[lastPos].classList.remove('top'); 98 | $bgs[pos].classList.add('visible'); 99 | $bgs[pos].classList.add('top'); 100 | 101 | // Hide last image after a short delay. 102 | window.setTimeout(function() { 103 | $bgs[lastPos].classList.remove('visible'); 104 | }, settings.delay / 2); 105 | 106 | }, settings.delay); 107 | 108 | })(); 109 | 110 | // Signup Form. 111 | (function() { 112 | 113 | // Vars. 114 | var $form = document.querySelectorAll('#signup-form')[0], 115 | $submit = document.querySelectorAll('#signup-form input[type="submit"]')[0], 116 | $message; 117 | 118 | // Bail if addEventListener isn't supported. 119 | if (!('addEventListener' in $form)) 120 | return; 121 | 122 | // Message. 123 | $message = document.createElement('span'); 124 | $message.classList.add('message'); 125 | $form.appendChild($message); 126 | 127 | $message._show = function(type, text) { 128 | 129 | $message.innerHTML = text; 130 | $message.classList.add(type); 131 | $message.classList.add('visible'); 132 | 133 | window.setTimeout(function() { 134 | $message._hide(); 135 | }, 3000); 136 | 137 | }; 138 | 139 | $message._hide = function() { 140 | $message.classList.remove('visible'); 141 | }; 142 | 143 | // Events. 144 | // Note: If you're *not* using AJAX, get rid of this event listener. 145 | $form.addEventListener('submit', function(event) { 146 | 147 | event.stopPropagation(); 148 | event.preventDefault(); 149 | 150 | // Hide message. 151 | $message._hide(); 152 | 153 | // Disable submit. 154 | $submit.disabled = true; 155 | 156 | // Process form. 157 | // Note: Doesn't actually do anything yet (other than report back with a "thank you"), 158 | // but there's enough here to piece together a working AJAX submission call that does. 159 | window.setTimeout(function() { 160 | 161 | // Reset form. 162 | $form.reset(); 163 | 164 | // Enable submit. 165 | $submit.disabled = false; 166 | 167 | // Show message. 168 | $message._show('success', 'Thank you! We\'ll response you soon'); 169 | //$message._show('failure', 'Something went wrong. Please try again.'); 170 | 171 | }, 750); 172 | 173 | }); 174 | 175 | })(); 176 | 177 | })(); -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/base/_bg.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* BG */ 12 | 13 | #bg { 14 | @include vendor('transition', 'opacity #{_duration(bg-fadein)} ease-in-out'); 15 | height: 100%; 16 | left: 0; 17 | opacity: 0.25; 18 | position: fixed; 19 | top: 0; 20 | width: 100%; 21 | z-index: 1; 22 | 23 | div { 24 | @include vendor('transition', ('opacity #{_duration(bg-transition)} ease', 'visibility #{_duration(bg-transition)}')); 25 | background-size: cover; 26 | height: 100%; 27 | left: 0; 28 | opacity: 0; 29 | position: absolute; 30 | top: 0; 31 | visibility: hidden; 32 | width: 150%; 33 | 34 | &.visible { 35 | @include vendor('animation', 'bg #{_duration(bg-slide)} linear infinite'); 36 | opacity: 1; 37 | visibility: visible; 38 | z-index: 1; 39 | 40 | &.top { 41 | z-index: 2; 42 | } 43 | 44 | @include breakpoint(large) { 45 | @include vendor('animation', 'bg #{_duration(bg-slide) * 0.65} linear infinite'); 46 | } 47 | 48 | @include breakpoint(small) { 49 | @include vendor('animation', 'bg #{_duration(bg-slide) * 0.4} linear infinite'); 50 | } 51 | } 52 | 53 | &:only-child { 54 | @include vendor('animation-direction', 'alternate !important'); 55 | } 56 | } 57 | 58 | body.is-loading & { 59 | opacity: 0; 60 | } 61 | } 62 | 63 | @include keyframes(bg) { 64 | 0% { 65 | @include vendor('transform', 'translateX(0)'); 66 | } 67 | 68 | 100% { 69 | @include vendor('transform', 'translateX(-25%)'); 70 | } 71 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/base/_page.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Basic */ 12 | 13 | // MSIE: Required for IEMobile. 14 | @-ms-viewport { 15 | width: device-width; 16 | } 17 | 18 | // MSIE: Prevents scrollbar from overlapping content. 19 | body { 20 | -ms-overflow-style: scrollbar; 21 | } 22 | 23 | // Ensures page width is always >=320px. 24 | @include breakpoint(xsmall) { 25 | html, body { 26 | min-width: 320px; 27 | } 28 | } 29 | 30 | html, body { 31 | height: 100%; 32 | overflow-x: hidden; 33 | width: 100%; 34 | 35 | @include breakpoint(short) { 36 | height: auto; 37 | min-height: 100%; 38 | } 39 | } 40 | 41 | body { 42 | @include vendor('display', 'flex'); 43 | @include vendor('flex-direction', 'column'); 44 | @include vendor('justify-content', 'center'); 45 | background-color: _palette(bg); 46 | padding: 6em 4em 4em 4em; 47 | 48 | > * { 49 | position: relative; 50 | z-index: 2; 51 | } 52 | 53 | &.is-loading { 54 | *, *:before, *:after { 55 | @include vendor('animation', 'none !important'); 56 | @include vendor('transition', 'none !important'); 57 | } 58 | } 59 | 60 | @include breakpoint(xlarge) { 61 | padding: 6em 3.5em 3.5em 3.5em; 62 | } 63 | 64 | @include breakpoint(small) { 65 | padding: 5em 2em 2em 2em; 66 | } 67 | 68 | @include breakpoint(xxsmall) { 69 | padding: 5em 1.25em 1.25em 1.25em; 70 | } 71 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/base/_typography.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Type */ 12 | 13 | body, input, select, textarea { 14 | color: _palette(fg); 15 | font-family: _font(family); 16 | font-size: 16pt; 17 | font-weight: _font(weight); 18 | letter-spacing: _font(letter-spacing); 19 | line-height: 1.65em; 20 | 21 | @include breakpoint(xlarge) { 22 | font-size: 12pt; 23 | } 24 | 25 | @include breakpoint(large) { 26 | font-size: 11pt; 27 | } 28 | 29 | @include breakpoint(medium) { 30 | font-size: 12pt; 31 | } 32 | 33 | @include breakpoint(small) { 34 | font-size: 12pt; 35 | } 36 | 37 | @include breakpoint(xsmall) { 38 | font-size: 12pt; 39 | } 40 | } 41 | 42 | a { 43 | @include vendor('transition', ( 44 | 'border-bottom-color #{_duration(transition)} ease', 45 | 'color #{_duration(transition)} ease' 46 | )); 47 | border-bottom: dotted 1px _palette(border2); 48 | color: _palette(accent, bg); 49 | text-decoration: none; 50 | 51 | &:hover { 52 | border-bottom-color: transparent; 53 | color: _palette(accent, bg) !important; 54 | text-decoration: none; 55 | } 56 | } 57 | 58 | strong, b { 59 | color: _palette(fg-bold); 60 | font-weight: _font(weight-bold); 61 | } 62 | 63 | em, i { 64 | font-style: italic; 65 | } 66 | 67 | p { 68 | margin: 0 0 _size(element-margin) 0; 69 | } 70 | 71 | h1, h2, h3, h4, h5, h6 { 72 | color: _palette(fg-bold); 73 | font-weight: _font(weight-bold); 74 | line-height: 1em; 75 | margin: 0 0 (_size(element-margin) * 0.5) 0; 76 | 77 | a { 78 | color: inherit; 79 | text-decoration: none; 80 | } 81 | } 82 | 83 | h1 { 84 | font-size: 2.5em; 85 | line-height: 1.25em; 86 | } 87 | 88 | h2 { 89 | font-size: 1.75em; 90 | line-height: 1.5em; 91 | } 92 | 93 | h3 { 94 | font-size: 1.35em; 95 | line-height: 1.5em; 96 | } 97 | 98 | h4 { 99 | font-size: 1.1em; 100 | line-height: 1.5em; 101 | } 102 | 103 | h5 { 104 | font-size: 0.9em; 105 | line-height: 1.5em; 106 | } 107 | 108 | h6 { 109 | font-size: 0.7em; 110 | line-height: 1.5em; 111 | } 112 | 113 | sub { 114 | font-size: 0.8em; 115 | position: relative; 116 | top: 0.5em; 117 | } 118 | 119 | sup { 120 | font-size: 0.8em; 121 | position: relative; 122 | top: -0.5em; 123 | } 124 | 125 | blockquote { 126 | border-left: solid (_size(border-width) * 4) _palette(border); 127 | font-style: italic; 128 | margin: 0 0 _size(element-margin) 0; 129 | padding: (_size(element-margin) / 4) 0 (_size(element-margin) / 4) _size(element-margin); 130 | } 131 | 132 | code { 133 | background: _palette(border-bg); 134 | border-radius: _size(border-radius); 135 | border: solid _size(border-width) _palette(border); 136 | font-family: _font(family-fixed); 137 | font-size: 0.9em; 138 | margin: 0 0.25em; 139 | padding: 0.25em 0.65em; 140 | } 141 | 142 | pre { 143 | -webkit-overflow-scrolling: touch; 144 | font-family: _font(family-fixed); 145 | font-size: 0.9em; 146 | margin: 0 0 _size(element-margin) 0; 147 | 148 | code { 149 | display: block; 150 | line-height: 1.75em; 151 | padding: 1em 1.5em; 152 | overflow-x: auto; 153 | } 154 | } 155 | 156 | hr { 157 | border: 0; 158 | border-bottom: solid _size(border-width) _palette(border); 159 | margin: _size(element-margin) 0; 160 | 161 | &.major { 162 | margin: (_size(element-margin) * 1.5) 0; 163 | } 164 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/components/_button.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Button */ 12 | 13 | input[type="submit"], 14 | input[type="reset"], 15 | input[type="button"], 16 | button, 17 | .button { 18 | @include vendor('appearance', 'none'); 19 | @include vendor('transition', ( 20 | 'background-color #{_duration(transition)} ease-in-out', 21 | 'color #{_duration(transition)} ease-in-out', 22 | 'opacity #{_duration(transition)} ease-in-out' 23 | )); 24 | background-color: _palette(accent, bg); 25 | border-radius: _size(border-radius); 26 | border: 0; 27 | color: _palette(accent, fg-bold) !important; 28 | cursor: pointer; 29 | display: inline-block; 30 | font-weight: _font(weight-bold); 31 | height: _size(element-height); 32 | line-height: _size(element-height); 33 | padding: 0 1.125em; 34 | text-align: center; 35 | text-decoration: none; 36 | white-space: nowrap; 37 | 38 | &:hover { 39 | background-color: lighten(_palette(accent, bg), 5); 40 | } 41 | 42 | &:active { 43 | background-color: darken(_palette(accent, bg), 5); 44 | } 45 | 46 | &.disabled, 47 | &:disabled { 48 | opacity: 0.5; 49 | } 50 | 51 | @include breakpoint(xsmall) { 52 | padding: 0; 53 | } 54 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/components/_form.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Form */ 12 | 13 | form { 14 | margin: 0 0 _size(element-margin) 0; 15 | 16 | .message { 17 | @include icon; 18 | @include vendor('transition', ( 19 | 'opacity #{_duration(transition)} ease-in-out', 20 | 'transform #{_duration(transition)} ease-in-out' 21 | )); 22 | @include vendor('transform', 'scale(1.05)'); 23 | height: _size(element-height); 24 | line-height: _size(element-height); 25 | opacity: 0; 26 | 27 | &:before { 28 | margin-right: 0.5em; 29 | } 30 | 31 | &.visible { 32 | @include vendor('transform', 'scale(1)'); 33 | opacity: 1; 34 | } 35 | 36 | &.success { 37 | color: _palette(positive, bg); 38 | 39 | &:before { 40 | content: '\f00c'; 41 | } 42 | } 43 | 44 | &.failure { 45 | color: _palette(negative, bg); 46 | 47 | &:before { 48 | content: '\f119'; 49 | } 50 | } 51 | } 52 | } 53 | 54 | label { 55 | color: _palette(fg-bold); 56 | display: block; 57 | font-size: 0.9em; 58 | font-weight: _font(weight-bold); 59 | margin: 0 0 (_size(element-margin) * 0.5) 0; 60 | } 61 | 62 | @include keyframes(focus) { 63 | 0% { @include vendor('transform', 'scale(1)'); } 64 | 50% { @include vendor('transform', 'scale(1.025)'); } 65 | 100% { @include vendor('transform', 'scale(1)'); } 66 | } 67 | 68 | input[type="text"], 69 | input[type="password"], 70 | input[type="email"], 71 | select, 72 | textarea { 73 | @include vendor('appearance', 'none'); 74 | @include vendor('transform', 'scale(1)'); 75 | @include vendor('transition', ( 76 | 'border-color #{_duration(transition)} ease', 77 | 'background-color #{_duration(transition)} ease' 78 | )); 79 | background-color: transparent; 80 | border-radius: _size(border-radius); 81 | border: none; 82 | border: solid _size(border-width) _palette(border); 83 | color: inherit; 84 | display: block; 85 | outline: 0; 86 | padding: 0 1em; 87 | text-decoration: none; 88 | width: 100%; 89 | 90 | &:invalid { 91 | box-shadow: none; 92 | } 93 | 94 | &:focus { 95 | @include vendor('animation', 'focus 0.1s'); 96 | background-color: _palette(border-bg); 97 | border-color: _palette(accent, bg); 98 | } 99 | } 100 | 101 | .select-wrapper { 102 | @include icon; 103 | display: block; 104 | position: relative; 105 | 106 | &:before { 107 | color: _palette(border); 108 | content: '\f078'; 109 | display: block; 110 | height: _size(element-height); 111 | line-height: _size(element-height); 112 | pointer-events: none; 113 | position: absolute; 114 | right: 0; 115 | text-align: center; 116 | top: 0; 117 | width: _size(element-height); 118 | } 119 | 120 | select::-ms-expand { 121 | display: none; 122 | } 123 | } 124 | 125 | input[type="text"], 126 | input[type="password"], 127 | input[type="email"], 128 | select { 129 | height: _size(element-height); 130 | } 131 | 132 | textarea { 133 | padding: 0.75em 1em; 134 | } 135 | 136 | input[type="checkbox"], 137 | input[type="radio"], { 138 | @include vendor('appearance', 'none'); 139 | display: block; 140 | float: left; 141 | margin-right: -2em; 142 | opacity: 0; 143 | width: 1em; 144 | z-index: -1; 145 | 146 | & + label { 147 | @include icon; 148 | color: _palette(fg); 149 | cursor: pointer; 150 | display: inline-block; 151 | font-size: 1em; 152 | font-weight: _font(weight); 153 | padding-left: (_size(element-height) * 0.6) + 0.75em; 154 | padding-right: 0.75em; 155 | position: relative; 156 | 157 | &:before { 158 | background: _palette(border-bg); 159 | border-radius: _size(border-radius); 160 | border: solid _size(border-width) _palette(border); 161 | content: ''; 162 | display: inline-block; 163 | height: (_size(element-height) * 0.6); 164 | left: 0; 165 | line-height: (_size(element-height) * 0.575); 166 | position: absolute; 167 | text-align: center; 168 | top: 0; 169 | width: (_size(element-height) * 0.6); 170 | } 171 | } 172 | 173 | &:checked + label { 174 | &:before { 175 | background: _palette(accent, bg); 176 | border-color: _palette(accent, bg); 177 | color: _palette(accent, fg-bold); 178 | content: '\f00c'; 179 | } 180 | } 181 | 182 | &:focus + label { 183 | &:before { 184 | border-color: _palette(accent, bg); 185 | box-shadow: 0 0 0 _size(border-width) _palette(accent, bg); 186 | } 187 | } 188 | } 189 | 190 | input[type="checkbox"] { 191 | & + label { 192 | &:before { 193 | border-radius: _size(border-radius); 194 | } 195 | } 196 | } 197 | 198 | input[type="radio"] { 199 | & + label { 200 | &:before { 201 | border-radius: 100%; 202 | } 203 | } 204 | } 205 | 206 | ::-webkit-input-placeholder { 207 | color: _palette(fg-light) !important; 208 | opacity: 1.0; 209 | } 210 | 211 | :-moz-placeholder { 212 | color: _palette(fg-light) !important; 213 | opacity: 1.0; 214 | } 215 | 216 | ::-moz-placeholder { 217 | color: _palette(fg-light) !important; 218 | opacity: 1.0; 219 | } 220 | 221 | :-ms-input-placeholder { 222 | color: _palette(fg-light) !important; 223 | opacity: 1.0; 224 | } 225 | 226 | .formerize-placeholder { 227 | color: _palette(fg-light) !important; 228 | opacity: 1.0; 229 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/components/_icon.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Icon */ 12 | 13 | .icon { 14 | @include icon; 15 | border-bottom: none; 16 | position: relative; 17 | 18 | > .label { 19 | display: none; 20 | } 21 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/components/_list.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* List */ 12 | 13 | ol { 14 | list-style: decimal; 15 | margin: 0 0 _size(element-margin) 0; 16 | padding-left: 1.25em; 17 | 18 | li { 19 | padding-left: 0.25em; 20 | } 21 | } 22 | 23 | ul { 24 | list-style: disc; 25 | margin: 0 0 _size(element-margin) 0; 26 | padding-left: 1em; 27 | 28 | li { 29 | padding-left: 0.5em; 30 | } 31 | 32 | &.icons { 33 | cursor: default; 34 | list-style: none; 35 | padding-left: 0; 36 | 37 | li { 38 | display: inline-block; 39 | padding: 0 1em 0 0; 40 | 41 | &:last-child { 42 | padding-right: 0; 43 | } 44 | 45 | .icon { 46 | &:before { 47 | font-size: 1.25em; 48 | } 49 | } 50 | 51 | a { 52 | color: inherit; 53 | } 54 | } 55 | } 56 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/components/_section.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Section/Article */ 12 | 13 | section, article { 14 | &.special { 15 | text-align: center; 16 | } 17 | } 18 | 19 | header { 20 | p { 21 | color: _palette(fg-light); 22 | position: relative; 23 | margin: 0 0 (_size(element-margin) * 0.75) 0; 24 | } 25 | 26 | h2 + p { 27 | font-size: 1.25em; 28 | margin-top: (_size(element-margin) * -0.5); 29 | line-height: 1.5em; 30 | } 31 | 32 | h3 + p { 33 | font-size: 1.1em; 34 | margin-top: (_size(element-margin) * -0.4); 35 | line-height: 1.5em; 36 | } 37 | 38 | h4 + p, 39 | h5 + p, 40 | h6 + p { 41 | font-size: 0.9em; 42 | margin-top: (_size(element-margin) * -0.3); 43 | line-height: 1.5em; 44 | } 45 | 46 | @include breakpoint(medium) { 47 | br { 48 | display: none; 49 | } 50 | } 51 | 52 | @include breakpoint(small) { 53 | br { 54 | display: inline; 55 | } 56 | } 57 | 58 | @include breakpoint(xsmall) { 59 | br { 60 | display: none; 61 | } 62 | } 63 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/ie8.scss: -------------------------------------------------------------------------------- 1 | @import 'libs/vars'; 2 | @import 'libs/functions'; 3 | @import 'libs/mixins'; 4 | @import 'libs/skel'; 5 | 6 | /* 7 | Eventually by HTML5 UP 8 | html5up.net | @ajlkn 9 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 10 | */ 11 | 12 | /* BG */ 13 | 14 | #bg { 15 | -ms-filter: "progid:DXImageTransform.Microsoft.Alpha(Opacity=25)"; 16 | } 17 | 18 | /* Type */ 19 | 20 | body, input, select, textarea { 21 | color: _palette(fg-bold); 22 | } 23 | 24 | /* Form */ 25 | 26 | input[type="text"], 27 | input[type="password"], 28 | input[type="email"], 29 | select, 30 | textarea { 31 | border: solid _size(border-width) _palette(fg-bold); 32 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/ie9.scss: -------------------------------------------------------------------------------- 1 | @import 'libs/vars'; 2 | @import 'libs/functions'; 3 | @import 'libs/mixins'; 4 | @import 'libs/skel'; 5 | 6 | /* 7 | Eventually by HTML5 UP 8 | html5up.net | @ajlkn 9 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 10 | */ 11 | 12 | /* Signup Form */ 13 | 14 | #signup-form { 15 | > * { 16 | display: inline-block; 17 | vertical-align: top; 18 | } 19 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/layout/_footer.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Footer */ 12 | 13 | #footer { 14 | @include vendor('transition', 'opacity 0.5s ease-in-out'); 15 | bottom: 4em; 16 | color: _palette(fg-light); 17 | left: 4em; 18 | opacity: 0.5; 19 | position: absolute; 20 | 21 | .icons { 22 | margin: 0 0 (_size(element-margin) * 0.25) 0; 23 | } 24 | 25 | .copyright { 26 | font-size: 0.8em; 27 | list-style: none; 28 | padding: 0; 29 | 30 | li { 31 | border-left: solid 1px _palette(border2); 32 | display: inline-block; 33 | line-height: 1em; 34 | margin: 0 0 0 0.75em; 35 | padding: 0 0 0 0.75em; 36 | 37 | &:first-child { 38 | border-left: 0; 39 | margin-left: 0; 40 | padding-left: 0; 41 | } 42 | } 43 | 44 | a { 45 | color: inherit; 46 | } 47 | } 48 | 49 | &:hover { 50 | opacity: 1; 51 | } 52 | 53 | > :last-child { 54 | margin-bottom: 0; 55 | } 56 | 57 | @include breakpoint(xlarge) { 58 | bottom: 3.5em; 59 | left: 3.5em; 60 | } 61 | 62 | @include breakpoint(small) { 63 | bottom: 2em; 64 | left: 2em; 65 | } 66 | 67 | @include breakpoint(xxsmall) { 68 | bottom: 1.25em; 69 | left: 1.25em; 70 | } 71 | 72 | @include breakpoint(short) { 73 | bottom: auto; 74 | left: auto; 75 | margin: (_size(element-margin) * 0.5) 0 0 0; 76 | position: relative; 77 | } 78 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/layout/_header.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Header */ 12 | 13 | #header { 14 | h1 { 15 | font-size: 3.25em; 16 | margin: 0 0 (_size(element-margin) * 0.275) 0; 17 | } 18 | 19 | p { 20 | font-size: 1.35em; 21 | line-height: 1.65em; 22 | } 23 | 24 | a { 25 | color: inherit; 26 | } 27 | 28 | @include breakpoint(small) { 29 | h1 { 30 | font-size: 2em; 31 | } 32 | 33 | p { 34 | font-size: 1em; 35 | } 36 | } 37 | 38 | @include breakpoint(xsmall) { 39 | margin: 0 0 (_size(element-margin) * 0.5) 0; 40 | } 41 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/layout/_signup-form.scss: -------------------------------------------------------------------------------- 1 | @import '../libs/vars'; 2 | @import '../libs/functions'; 3 | @import '../libs/mixins'; 4 | 5 | /// 6 | /// Eventually by HTML5 UP 7 | /// html5up.net | @ajlkn 8 | /// Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 9 | /// 10 | 11 | /* Signup Form */ 12 | 13 | #signup-form { 14 | @include vendor('display', 'flex'); 15 | position: relative; 16 | 17 | input[type="text"], 18 | input[type="password"], 19 | input[type="email"] { 20 | width: 18em; 21 | } 22 | 23 | > * { 24 | margin: 0 0 0 1em; 25 | } 26 | 27 | > :first-child { 28 | margin: 0 0 0 0; 29 | } 30 | 31 | @include breakpoint(xsmall) { 32 | @include vendor('flex-direction', 'column'); 33 | 34 | input[type="type"], 35 | input[type="password"], 36 | input[type="email"] { 37 | width: 100%; 38 | } 39 | 40 | > * { 41 | margin: 1.25em 0 0 0; 42 | } 43 | 44 | .message { 45 | bottom: -1.5em; 46 | font-size: 0.9em; 47 | height: 1em; 48 | left: 0; 49 | line-height: inherit; 50 | margin-top: 0; 51 | position: absolute; 52 | } 53 | } 54 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/libs/_functions.scss: -------------------------------------------------------------------------------- 1 | /// Gets a duration value. 2 | /// @param {string} $keys Key(s). 3 | /// @return {string} Value. 4 | @function _duration($keys...) { 5 | @return val($duration, $keys...); 6 | } 7 | 8 | /// Gets a font value. 9 | /// @param {string} $keys Key(s). 10 | /// @return {string} Value. 11 | @function _font($keys...) { 12 | @return val($font, $keys...); 13 | } 14 | 15 | /// Gets a misc value. 16 | /// @param {string} $keys Key(s). 17 | /// @return {string} Value. 18 | @function _misc($keys...) { 19 | @return val($misc, $keys...); 20 | } 21 | 22 | /// Gets a palette value. 23 | /// @param {string} $keys Key(s). 24 | /// @return {string} Value. 25 | @function _palette($keys...) { 26 | @return val($palette, $keys...); 27 | } 28 | 29 | /// Gets a size value. 30 | /// @param {string} $keys Key(s). 31 | /// @return {string} Value. 32 | @function _size($keys...) { 33 | @return val($size, $keys...); 34 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/libs/_mixins.scss: -------------------------------------------------------------------------------- 1 | /// Makes an element's :before pseudoelement a FontAwesome icon. 2 | /// @param {string} $content Optional content value to use. 3 | /// @param {string} $where Optional pseudoelement to target (before or after). 4 | @mixin icon($content: false, $where: before) { 5 | 6 | text-decoration: none; 7 | 8 | &:#{$where} { 9 | 10 | @if $content { 11 | content: $content; 12 | } 13 | 14 | -moz-osx-font-smoothing: grayscale; 15 | -webkit-font-smoothing: antialiased; 16 | font-family: FontAwesome; 17 | font-style: normal; 18 | font-weight: normal; 19 | text-transform: none !important; 20 | 21 | } 22 | 23 | } 24 | 25 | /// Applies padding to an element, taking the current element-margin value into account. 26 | /// @param {mixed} $tb Top/bottom padding. 27 | /// @param {mixed} $lr Left/right padding. 28 | /// @param {list} $pad Optional extra padding (in the following order top, right, bottom, left) 29 | /// @param {bool} $important If true, adds !important. 30 | @mixin padding($tb, $lr, $pad: (0,0,0,0), $important: null) { 31 | 32 | @if $important { 33 | $important: '!important'; 34 | } 35 | 36 | padding: ($tb + nth($pad,1)) ($lr + nth($pad,2)) max(0.1em, $tb - _size(element-margin) + nth($pad,3)) ($lr + nth($pad,4)) #{$important}; 37 | 38 | } 39 | 40 | /// Encodes a SVG data URL so IE doesn't choke (via codepen.io/jakob-e/pen/YXXBrp). 41 | /// @param {string} $svg SVG data URL. 42 | /// @return {string} Encoded SVG data URL. 43 | @function svg-url($svg) { 44 | 45 | $svg: str-replace($svg, '"', '\''); 46 | $svg: str-replace($svg, '<', '%3C'); 47 | $svg: str-replace($svg, '>', '%3E'); 48 | $svg: str-replace($svg, '&', '%26'); 49 | $svg: str-replace($svg, '#', '%23'); 50 | $svg: str-replace($svg, '{', '%7B'); 51 | $svg: str-replace($svg, '}', '%7D'); 52 | $svg: str-replace($svg, ';', '%3B'); 53 | 54 | @return url("data:image/svg+xml;charset=utf8,#{$svg}"); 55 | 56 | } -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/libs/_vars.scss: -------------------------------------------------------------------------------- 1 | // Misc. 2 | $misc: ( 3 | ); 4 | 5 | // Duration. 6 | $duration: ( 7 | nav: 0.5s, 8 | transition: 0.2s, 9 | bg-fadein: 2s, 10 | bg-transition: 3s, 11 | bg-slide: 45s // (lower = faster, higher = slower) 12 | ); 13 | 14 | // Size. 15 | $size: ( 16 | border-radius: 6px, 17 | border-width: 2px, 18 | element-height: 2.75em, 19 | element-margin: 2em 20 | ); 21 | 22 | // Font. 23 | $font: ( 24 | family: ('Roboto', sans-serif), 25 | family-fixed: ('Courier New', monospace), 26 | weight: 400, 27 | weight-bold: 700, 28 | letter-spacing: -0.01em 29 | ); 30 | 31 | // Palette. 32 | $palette: ( 33 | bg: #000, 34 | fg: rgba(255,255,255,0.75), 35 | fg-bold: #fff, 36 | fg-light: rgba(255,255,255,0.5), 37 | border: rgba(255,255,255,0.35), 38 | border-bg: rgba(255,255,255,0.125), 39 | border2: rgba(255,255,255,0.25), 40 | 41 | accent: ( 42 | bg: #1cb495, 43 | fg: mix(#1cb495, #ffffff, 25%), 44 | fg-bold: #ffffff, 45 | fg-light: mix(#1cb495, #ffffff, 40%), 46 | border: rgba(255,255,255,0.25), 47 | border-bg: rgba(255,255,255,0.075), 48 | ), 49 | 50 | positive: ( 51 | bg: #1cb495, 52 | fg: #ffffff 53 | ), 54 | 55 | negative: ( 56 | bg: #ff2361, 57 | fg: #ffffff 58 | ) 59 | ); -------------------------------------------------------------------------------- /src/main/resources/landing/assets/sass/main.scss: -------------------------------------------------------------------------------- 1 | @import 'libs/vars'; 2 | @import 'libs/functions'; 3 | @import 'libs/mixins'; 4 | @import 'libs/skel'; 5 | @import 'font-awesome.min.css'; 6 | @import url('https://fonts.googleapis.com/css?family=Roboto:400,700'); 7 | 8 | /* 9 | Eventually by HTML5 UP 10 | html5up.net | @ajlkn 11 | Free for personal and commercial use under the CCA 3.0 license (html5up.net/license) 12 | */ 13 | 14 | @include skel-breakpoints(( 15 | xlarge: '(max-width: 1680px)', 16 | large: '(max-width: 1280px)', 17 | medium: '(max-width: 980px)', 18 | small: '(max-width: 736px)', 19 | xsmall: '(max-width: 480px)', 20 | xxsmall: '(max-width: 360px)', 21 | short: '(max-height: 640px)' 22 | )); 23 | 24 | @include skel-layout(( 25 | reset: 'full', 26 | boxModel: 'border' 27 | )); 28 | 29 | // Base. 30 | 31 | @import 'base/page'; 32 | @import 'base/bg'; 33 | @import 'base/typography'; 34 | 35 | // Components. 36 | 37 | @import 'components/section'; 38 | @import 'components/icon'; 39 | @import 'components/list'; 40 | @import 'components/form'; 41 | @import 'components/button'; 42 | 43 | // Layout. 44 | 45 | @import 'layout/header'; 46 | @import 'layout/signup-form'; 47 | @import 'layout/footer'; -------------------------------------------------------------------------------- /src/main/resources/landing/images/bg02.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/images/bg02.jpg -------------------------------------------------------------------------------- /src/main/resources/landing/images/bg03.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/images/bg03.jpg -------------------------------------------------------------------------------- /src/main/resources/landing/images/bg04.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/images/bg04.jpg -------------------------------------------------------------------------------- /src/main/resources/landing/images/bg05.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/images/bg05.jpg -------------------------------------------------------------------------------- /src/main/resources/landing/images/bg06.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/datafibers-community/df_data_service/8bb6ed329e5f3404c8e3d614a62d1ed62d67b443/src/main/resources/landing/images/bg06.jpg -------------------------------------------------------------------------------- /src/main/resources/landing/index.html: -------------------------------------------------------------------------------- 1 | 2 | 7 | 8 | 9 | Welcome to DataFibers 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 24 | 25 |

26 | 27 | 28 | 29 | 30 |

31 | 32 | 38 | 39 | 40 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Root logger option 2 | # log4j.rootLogger=ERROR,stdout,file,MongoDB 3 | log4j.rootLogger=ERROR,stdout,file 4 | 5 | # Package Logger 6 | log4j.logger.io.vertx.core.impl.BlockedThreadChecker=ERROR 7 | log4j.logger.com.datafibers.service.DFInitService=INFO 8 | log4j.logger.com.datafibers.service.DFWebUI=INFO 9 | log4j.logger.com.datafibers.service.DFDataProcessor=INFO 10 | log4j.logger.com.datafibers.processor.ProcessorStreamBack=INFO 11 | log4j.logger.com.datafibers.processor.ProcessorConnectKafka=INFO 12 | log4j.logger.com.datafibers.processor.ProcessorTopicSchemaRegistry=INFO 13 | log4j.logger.com.datafibers.processor.ProcessorTransformFlink=INFO 14 | log4j.logger.com.datafibers.processor.ProcessorTransformSpark=INFO 15 | 16 | # Direct log messages to stdout 17 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 18 | log4j.appender.stdout.Target=System.out 19 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 20 | log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} [%-5p] %-23c{1}:%-4L - %m%n 21 | 22 | # Redirect log messages to a log file, support file rolling. 23 | log4j.appender.file=org.apache.log4j.RollingFileAppender 24 | log4j.appender.file.File=/home/vagrant/log4j-application.log 25 | log4j.appender.file.MaxFileSize=5MB 26 | log4j.appender.file.MaxBackupIndex=10 27 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 28 | log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} [%-5p] %-23c{1}:%-4L - %m%n 29 | 30 | # Direct log messages to mongodb 31 | # log4j.appender.MongoDB=org.log4mongo.MongoDbAppender 32 | # log4j.appender.MongoDB.databaseName=DEFAULT_DB 33 | # log4j.appender.MongoDB.collectionName=df_log 34 | # log4j.appender.MongoDB.hostname=localhost 35 | # log4j.appender.MongoDB.port=27017 36 | -------------------------------------------------------------------------------- /src/test/java/com/datafibers/service/MyFirstVerticleTest.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.service; 2 | 3 | import com.datafibers.model.DFJobPOPJ; 4 | import de.flapdoodle.embed.mongo.MongodExecutable; 5 | import de.flapdoodle.embed.mongo.MongodProcess; 6 | import de.flapdoodle.embed.mongo.MongodStarter; 7 | import de.flapdoodle.embed.mongo.config.IMongodConfig; 8 | import de.flapdoodle.embed.mongo.config.MongodConfigBuilder; 9 | import de.flapdoodle.embed.mongo.config.Net; 10 | import de.flapdoodle.embed.mongo.distribution.Version; 11 | import de.flapdoodle.embed.process.runtime.Network; 12 | import io.vertx.core.DeploymentOptions; 13 | import io.vertx.core.Vertx; 14 | import io.vertx.core.json.Json; 15 | import io.vertx.core.json.JsonObject; 16 | import io.vertx.ext.unit.Async; 17 | import io.vertx.ext.unit.TestContext; 18 | import io.vertx.ext.unit.junit.VertxUnitRunner; 19 | import org.junit.*; 20 | import org.junit.runner.RunWith; 21 | 22 | import java.io.IOException; 23 | import java.net.ServerSocket; 24 | 25 | /** 26 | * This is our JUnit test for our verticle. The test uses vertx-unit, so we declare a custom runner. 27 | */ 28 | @RunWith(VertxUnitRunner.class) 29 | public class MyFirstVerticleTest { 30 | 31 | private Vertx vertx; 32 | private Integer port; 33 | private static MongodProcess MONGO; 34 | private static int MONGO_PORT = 12345; 35 | 36 | @BeforeClass 37 | public static void initialize() throws IOException { 38 | MongodStarter starter = MongodStarter.getDefaultInstance(); 39 | 40 | IMongodConfig mongodConfig = new MongodConfigBuilder() 41 | .version(Version.Main.PRODUCTION) 42 | .net(new Net(MONGO_PORT, Network.localhostIsIPv6())) 43 | .build(); 44 | 45 | MongodExecutable mongodExecutable = starter.prepare(mongodConfig); 46 | MONGO = mongodExecutable.start(); 47 | } 48 | 49 | @AfterClass 50 | public static void shutdown() { 51 | MONGO.stop(); 52 | } 53 | 54 | /** 55 | * Before executing our test, let's deploy our verticle. 56 | *

57 | * This method instantiates a new Vertx and deploy the verticle. Then, it waits in the verticle has successfully 58 | * completed its start sequence (thanks to `context.asyncAssertSuccess`). 59 | * 60 | * @param context the test context. 61 | */ 62 | @Before 63 | public void setUp(TestContext context) throws IOException { 64 | vertx = Vertx.vertx(); 65 | 66 | // Let's configure the verticle to listen on the 'test' port (randomly picked). 67 | // We create deployment options and set the _configuration_ json object: 68 | ServerSocket socket = new ServerSocket(0); 69 | port = socket.getLocalPort(); 70 | socket.close(); 71 | 72 | DeploymentOptions options = new DeploymentOptions() 73 | .setConfig(new JsonObject() 74 | .put("http.port", port) 75 | .put("db_name", "whiskies-test") 76 | .put("connection_string", "mongodb://localhost:" + MONGO_PORT) 77 | ); 78 | 79 | // We pass the options as the second parameter of the deployVerticle method. 80 | vertx.deployVerticle(DFDataProcessor.class.getName(), options, context.asyncAssertSuccess()); 81 | } 82 | 83 | /** 84 | * This method, called after our test, just cleanup everything by closing the vert.x instance 85 | * 86 | * @param context the test context 87 | */ 88 | @After 89 | public void tearDown(TestContext context) { 90 | vertx.close(context.asyncAssertSuccess()); 91 | } 92 | 93 | /** 94 | * Let's ensure that our application behaves correctly. 95 | * 96 | * @param context the test context 97 | */ 98 | @Test 99 | public void testMyApplication(TestContext context) { 100 | // This test is asynchronous, so get an async handler to inform the test when we are done. 101 | final Async async = context.async(); 102 | 103 | // We create a HTTP client and query our application. When we get the response we check it contains the 'Hello' 104 | // message. Then, we call the `complete` method on the async handler to declare this async (and here the test) done. 105 | // Notice that the assertions are made on the 'context' object and are not Junit assert. This ways it manage the 106 | // async aspect of the test the right way. 107 | vertx.createHttpClient().getNow(port, "localhost", "/", response -> { 108 | response.handler(body -> { 109 | context.assertTrue(body.toString().contains("Hello")); 110 | async.complete(); 111 | }); 112 | }); 113 | } 114 | 115 | @Test 116 | public void checkThatTheIndexPageIsServed(TestContext context) { 117 | Async async = context.async(); 118 | vertx.createHttpClient().getNow(port, "localhost", "/assets/index.html", response -> { 119 | context.assertEquals(response.statusCode(), 200); 120 | context.assertEquals(response.headers().get("content-type"), "text/html"); 121 | response.bodyHandler(body -> { 122 | context.assertTrue(body.toString().contains("My DFJobPOPJ Collection")); 123 | async.complete(); 124 | }); 125 | }); 126 | } 127 | 128 | @Test 129 | public void checkThatWeCanAdd(TestContext context) { 130 | Async async = context.async(); 131 | final String json = Json.encodePrettily(new DFJobPOPJ("Jameson", "Ireland","Register")); 132 | vertx.createHttpClient().post(port, "localhost", "/api/df") 133 | .putHeader("content-type", "application/json") 134 | .putHeader("content-length", Integer.toString(json.length())) 135 | .handler(response -> { 136 | context.assertEquals(response.statusCode(), 201); 137 | context.assertTrue(response.headers().get("content-type").contains("application/json")); 138 | response.bodyHandler(body -> { 139 | final DFJobPOPJ DFJob = Json.decodeValue(body.toString(), DFJobPOPJ.class); 140 | context.assertEquals(DFJob.getName(), "Jameson"); 141 | context.assertEquals(DFJob.getConnectUid(), "Ireland"); 142 | context.assertNotNull(DFJob.getId()); 143 | async.complete(); 144 | }); 145 | }) 146 | .write(json) 147 | .end(); 148 | } 149 | } 150 | -------------------------------------------------------------------------------- /src/test/java/com/datafibers/service/MyRestIT.java: -------------------------------------------------------------------------------- 1 | package com.datafibers.service; 2 | 3 | import com.datafibers.model.DFJobPOPJ; 4 | import com.jayway.restassured.RestAssured; 5 | import org.junit.AfterClass; 6 | import org.junit.BeforeClass; 7 | import org.junit.Test; 8 | 9 | import static com.jayway.restassured.RestAssured.*; 10 | import static org.assertj.core.api.Assertions.assertThat; 11 | import static org.hamcrest.Matchers.equalTo; 12 | 13 | /** 14 | * These tests checks our REST API. 15 | */ 16 | public class MyRestIT { 17 | 18 | @BeforeClass 19 | public static void configureRestAssured() { 20 | RestAssured.baseURI = "http://localhost"; 21 | RestAssured.port = Integer.getInteger("http.port", 8082); 22 | } 23 | 24 | @AfterClass 25 | public static void unconfigureRestAssured() { 26 | RestAssured.reset(); 27 | } 28 | 29 | @Test 30 | public void checkThatWeCanRetrieveIndividualProduct() { 31 | // Get the list of bottles, ensure it's a success and extract the first id. 32 | final String id = get("/api/whiskies").then() 33 | .assertThat() 34 | .statusCode(200) 35 | .extract() 36 | .jsonPath().getString("find { it.name=='Bowmore 15 Years Laimrig' }.id"); 37 | 38 | // Now get the individual resource and check the content 39 | get("/api/whiskies/" + id).then() 40 | .assertThat() 41 | .statusCode(200) 42 | .body("name", equalTo("Bowmore 15 Years Laimrig")) 43 | .body("origin", equalTo("Scotland, Islay")) 44 | .body("id", equalTo(id)); 45 | } 46 | 47 | @Test 48 | public void checkWeCanAddAndDeleteAProduct() { 49 | // Create a new bottle and retrieve the result (as a DFJobPOPJ instance). 50 | DFJobPOPJ DFJob = given() 51 | .body("{\"name\":\"Jameson\", \"origin\":\"Ireland\"}").request().post("/api/whiskies").thenReturn().as(DFJobPOPJ.class); 52 | assertThat(DFJob.getName()).isEqualToIgnoringCase("Jameson"); 53 | assertThat(DFJob.getConnectUid()).isEqualToIgnoringCase("Ireland"); 54 | assertThat(DFJob.getId()).isNotEmpty(); 55 | 56 | 57 | 58 | // Check that it has created an individual resource, and check the content. 59 | get("/api/whiskies/" + DFJob.getId()).then() 60 | .assertThat() 61 | .statusCode(200) 62 | .body("name", equalTo("Jameson")) 63 | .body("origin", equalTo("Ireland")) 64 | .body("id", equalTo(DFJob.getId())); 65 | 66 | 67 | 68 | // Delete the bottle 69 | delete("/api/whiskies/" + DFJob.getId()).then().assertThat().statusCode(204); 70 | 71 | // Check that the resource is not available anymore 72 | get("/api/whiskies/" + DFJob.getId()).then() 73 | .assertThat() 74 | .statusCode(404); 75 | 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/test/resources/my-it-config.json: -------------------------------------------------------------------------------- 1 | { 2 | "http.port": ${http.port}, 3 | "db_name": "whiskies-it", 4 | "connection_string": "mongodb://localhost:37017" 5 | } --------------------------------------------------------------------------------