Repository: incubator-griffin Updated Branches: refs/heads/master 1b2b3e352 -> 3545a71c7
update measure documents Author: Lionel Liu <bhlx3l...@163.com> Closes #438 from bhlx3lyx7/doc-update. Project: http://git-wip-us.apache.org/repos/asf/incubator-griffin/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-griffin/commit/3545a71c Tree: http://git-wip-us.apache.org/repos/asf/incubator-griffin/tree/3545a71c Diff: http://git-wip-us.apache.org/repos/asf/incubator-griffin/diff/3545a71c Branch: refs/heads/master Commit: 3545a71c7c63623869f20aa414eb617fcb0786c0 Parents: 1b2b3e3 Author: Lionel Liu <bhlx3l...@163.com> Authored: Tue Oct 16 14:19:44 2018 +0800 Committer: William Guo <gu...@apache.org> Committed: Tue Oct 16 14:19:44 2018 +0800 ---------------------------------------------------------------------- griffin-doc/measure/dsl-guide.md | 6 +- griffin-doc/measure/measure-batch-sample.md | 14 +++-- .../measure/measure-configuration-guide.md | 55 ++++++++---------- griffin-doc/measure/measure-streaming-sample.md | 59 ++++++++++---------- .../configuration/dqdefinition/DQConfig.scala | 3 - 5 files changed, 66 insertions(+), 71 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/3545a71c/griffin-doc/measure/dsl-guide.md ---------------------------------------------------------------------- diff --git a/griffin-doc/measure/dsl-guide.md b/griffin-doc/measure/dsl-guide.md index 4eb294e..5296176 100644 --- a/griffin-doc/measure/dsl-guide.md +++ b/griffin-doc/measure/dsl-guide.md @@ -24,14 +24,14 @@ Griffin DSL is designed for DQ measurement, as a SQL-like language, which descri Griffin DSL syntax is easy to learn as it's SQL-like, case insensitive. ### Supporting process -- logical operation: `not, and, or, in, between, like, is null, is nan, =, !=, <>, <=, >=, <, >` +- logical operation: `not, and, or, in, between, like, rlike, is null, is nan, =, !=, <>, <=, >=, <, >` - mathematical operation: `+, -, *, /, %` - sql statement: `as, where, group by, having, order by, limit` ### Keywords - `null, nan, true, false` - `not, and, or` -- `in, between, like, is` +- `in, between, like, rlike, is` - `select, distinct, from, as, where, group, by, having, order, desc, asc, limit` ### Operators @@ -79,6 +79,8 @@ Griffin DSL syntax is easy to learn as it's SQL-like, case insensitive. e.g. `source.age between 3 and 30`, `source.age between (3, 30)` - **like**: like clause like sql. e.g. `source.name like "%abc%"` +- **rlike**: rlike clause like spark sql. + e.g. `source.name rlike "^abc.*$"` - **is null**: is null operator like sql. e.g. `source.desc is not null` - **is nan**: check if the value is not a number, the syntax like `is null` http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/3545a71c/griffin-doc/measure/measure-batch-sample.md ---------------------------------------------------------------------- diff --git a/griffin-doc/measure/measure-batch-sample.md b/griffin-doc/measure/measure-batch-sample.md index 1a9bc41..7867ea5 100644 --- a/griffin-doc/measure/measure-batch-sample.md +++ b/griffin-doc/measure/measure-batch-sample.md @@ -59,7 +59,7 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc { "dsl.type": "griffin-dsl", "dq.type": "accuracy", - "name": "accu", + "out.dataframe.name": "accu", "rule": "source.user_id = target.user_id AND upper(source.first_name) = upper(target.first_name) AND source.last_name = target.last_name AND source.address = target.address AND source.email = target.email AND source.phone = target.phone AND source.post_code = target.post_code", "details": { "source": "source", @@ -80,7 +80,9 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc ] } ] - } + }, + + "sinks": ["CONSOLE", "ELASTICSEARCH"] } ``` Above is the configure file of batch accuracy job. @@ -121,7 +123,7 @@ The miss records of source will be persisted as record. { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "name": "prof", + "out.dataframe.name": "prof", "rule": "select max(age) as `max_age`, min(age) as `min_age` from source", "out": [ { @@ -133,7 +135,7 @@ The miss records of source will be persisted as record. { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "name": "name_grp", + "out.dataframe.name": "name_grp", "rule": "select name, count(*) as cnt from source group by name", "out": [ { @@ -144,7 +146,9 @@ The miss records of source will be persisted as record. ] } ] - } + }, + + "sinks": ["CONSOLE", "ELASTICSEARCH"] } ``` Above is the configure file of batch profiling job. http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/3545a71c/griffin-doc/measure/measure-configuration-guide.md ---------------------------------------------------------------------- diff --git a/griffin-doc/measure/measure-configuration-guide.md b/griffin-doc/measure/measure-configuration-guide.md index 13774e1..43cb8ac 100644 --- a/griffin-doc/measure/measure-configuration-guide.md +++ b/griffin-doc/measure/measure-configuration-guide.md @@ -43,7 +43,7 @@ Apache Griffin measure module needs two configuration files to define the parame "sinks": [ { - "type": "log", + "type": "console", "config": { "max.log.lines": 100 } @@ -56,7 +56,7 @@ Apache Griffin measure module needs two configuration files to define the parame } ], - "info.cache": [ + "griffin.checkpoint": [ { "type": "zk", "config": { @@ -79,30 +79,30 @@ Above lists environment parameters. + batch.interval: Interval of dumping streaming data, for streaming mode. + process.interval: Interval of processing dumped streaming data, for streaming mode. + config: Configuration of spark parameters. -- **persist**: This field configures list of metrics persist parameters, multiple persist ways are supported. Details of persist configuration [here](#persist). -- **info.cache**: This field configures list of information cache parameters, multiple cache ways are supported. It is only for streaming dq case. Details of info cache configuration [here](#info-cache). +- **sinks**: This field configures list of metrics sink parameters, multiple sink ways are supported. Details of sink configuration [here](#sinks). +- **griffin.checkpoint**: This field configures list of griffin checkpoint parameters, multiple cache ways are supported. It is only for streaming dq case. Details of info cache configuration [here](#griffin-checkpoint). ### <a name="sinks"></a>Sinks -- **type**: Metrics and records persist type, "log", "hdfs", "http", "mongo". -- **config**: Configure parameters of each persist type. - + log persist (aliases: "console") +- **type**: Metrics and records sink type, "console", "hdfs", "http", "mongo". +- **config**: Configure parameters of each sink type. + + console sink (aliases: "log") * max.log.lines: the max lines of log. - + hdfs persist - * path: hdfs path to persist metrics - * max.persist.lines: the max lines of total persist data. - * max.lines.per.file: the max lines of each persist file. - + http persist (aliases: "es", "elasticsearch") - * api: api to submit persist metrics. + + hdfs sink + * path: hdfs path to sink metrics + * max.persist.lines: the max lines of total sink data. + * max.lines.per.file: the max lines of each sink file. + + http sink (aliases: "es", "elasticsearch") + * api: api to submit sink metrics. * method: http method, "post" default. - + mongo persist + + mongo sink * url: url of mongo db. * database: database name. * collection: collection name. -### <a name="info-cache"></a>Info Cache -- **type**: Information cache type, "zk" for zookeeper cache. -- **config**: Configure parameters of info cache type. - + zookeeper cache +### <a name="griffin-checkpoint"></a>Griffin Checkpoint +- **type**: Griffin checkpoint type, "zk" for zookeeper checkpoint. +- **config**: Configure parameters of griffin checkpoint type. + + zookeeper checkpoint * hosts: zookeeper hosts list as a string, separated by comma. * namespace: namespace of cache info, "" as default. * lock.path: path of lock info, "lock" as default. @@ -150,7 +150,7 @@ Above lists environment parameters. { "dsl.type": "griffin-dsl", "dq.type": "accuracy", - "name": "accu", + "out.dataframe.name": "accu", "rule": "source.user_id = target.user_id AND upper(source.first_name) = upper(target.first_name) AND source.last_name = target.last_name AND source.address = target.address AND source.email = target.email AND source.phone = target.phone AND source.post_code = target.post_code", "details": { "source": "source", @@ -206,9 +206,10 @@ Above lists DQ job configure parameters. * done.file: ### <a name="rule"></a>Rule -- **dsl.type**: Rule dsl type, "spark-sql", "df-opr" and "griffin-dsl". +- **dsl.type**: Rule dsl type, "spark-sql", "df-ops" and "griffin-dsl". - **dq.type**: DQ type of this rule, only for "griffin-dsl" type. Supported types: "accuracy", "profiling", "timeliness", "uniqueness", "completeness". -- **name** (step information): Result table name of this rule, optional for "griffin-dsl" type. +- **out.dataframe.name** (step information): Output table name of this rule, could be used in the following rules. +- **in.dataframe.name** (step information): Input table name of this rule, only used for "df-ops" type. - **rule**: The rule string. - **details**: Details of this rule, optional. + accuracy dq type detail configuration @@ -219,14 +220,6 @@ Above lists DQ job configure parameters. * matched: the matched count name in metric, optional. + profiling dq type detail configuration * source: the data source name which as source in profiling, default is the name of first data source in "data.sources" if not configured. If the griffin-dsl rule contains from clause, this parameter is ignored. - + uniqueness dq type detail configuration - * source: name of data source to measure uniqueness. - * target: name of data source to compare with. It is always the same as source, or more than source. - * unique: the unique count name in metric, optional. - * total: the total count name in metric, optional. - * dup: the duplicate count name in metric, optional. - * num: the duplicate number name in metric, optional. - * duplication.array: optional, if set as a non-empty string, the duplication metric will be computed, and the group metric name is this string. + distinctness dq type detail configuration * source: name of data source to measure uniqueness. * target: name of data source to compare with. It is always the same as source, or more than source. @@ -241,7 +234,7 @@ Above lists DQ job configure parameters. * source: name of data source to measure timeliness. * latency: the latency column name in metric, optional. * threshold: optional, if set as a time string like "1h", the items with latency more than 1 hour will be record. -- **out**: Lits of output sinks for the job. +- **out**: List of output sinks for the job. + Metric output. * type: "metric" * name: Metric name, semantics depends on "flatten" field value. @@ -250,7 +243,7 @@ Above lists DQ job configure parameters. - entries: sends first row of data frame as metric results, like like `{"agg_col": "value"}` - array: wraps all metrics into a map, like `{"my_out_name": [{"agg_col": "value"}]}` - map: wraps first row of data frame into a map, like `{"my_out_name": {"agg_col": "value"}}` - + Record output. Currenly handled only by HDFS sink. + + Record output. Currently handled only by HDFS sink. * type: "record" * name: File name within sink output folder to dump files to. + Data source cache update for streaming jobs. http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/3545a71c/griffin-doc/measure/measure-streaming-sample.md ---------------------------------------------------------------------- diff --git a/griffin-doc/measure/measure-streaming-sample.md b/griffin-doc/measure/measure-streaming-sample.md index 1d9f70e..30ed718 100644 --- a/griffin-doc/measure/measure-streaming-sample.md +++ b/griffin-doc/measure/measure-streaming-sample.md @@ -48,17 +48,15 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc }, "pre.proc": [ { - "dsl.type": "df-opr", - "name": "${s1}", - "rule": "from_json", - "details": { - "df.name": "${this}" - } + "dsl.type": "df-ops", + "in.dataframe.name": "this", + "out.dataframe.name": "s1", + "rule": "from_json" }, { "dsl.type": "spark-sql", - "name": "${this}", - "rule": "select name, age from ${s1}" + "out.dataframe.name": "this", + "rule": "select name, age from s1" } ] } @@ -68,7 +66,8 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc "info.path": "source", "ready.time.interval": "10s", "ready.time.delay": "0", - "time.range": ["-2m", "0"] + "time.range": ["-2m", "0"], + "updatable": true } }, { "name": "target", @@ -89,17 +88,15 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc }, "pre.proc": [ { - "dsl.type": "df-opr", - "name": "${t1}", - "rule": "from_json", - "details": { - "df.name": "${this}" - } + "dsl.type": "df-ops", + "in.dataframe.name": "this", + "out.dataframe.name": "t1", + "rule": "from_json" }, { "dsl.type": "spark-sql", - "name": "${this}", - "rule": "select name, age from ${t1}" + "out.dataframe.name": "this", + "rule": "select name, age from t1" } ] } @@ -119,7 +116,7 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc { "dsl.type": "griffin-dsl", "dq.type": "accuracy", - "name": "accu", + "out.dataframe.name": "accu", "rule": "source.name = target.name and source.age = target.age", "details": { "source": "source", @@ -140,7 +137,9 @@ Apache Griffin measures consist of batch measure and streaming measure, this doc ] } ] - } + }, + + "sinks": ["CONSOLE","ELASTICSEARCH"] } ``` Above is the configure file of streaming accuracy job. @@ -199,17 +198,15 @@ The miss records of source will be persisted as record. }, "pre.proc": [ { - "dsl.type": "df-opr", - "name": "${s1}", - "rule": "from_json", - "details": { - "df.name": "${this}" - } + "dsl.type": "df-ops", + "in.dataframe.name": "this", + "out.dataframe.name": "s1", + "rule": "from_json" }, { "dsl.type": "spark-sql", - "name": "${this}", - "rule": "select name, age from ${s1}" + "out.dataframe.name": "this", + "rule": "select name, age from s1" } ] } @@ -229,7 +226,7 @@ The miss records of source will be persisted as record. { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "name": "prof", + "out.dataframe.name": "prof", "rule": "select count(name) as `cnt`, max(age) as `max`, min(age) as `min` from source", "out": [ { @@ -241,7 +238,7 @@ The miss records of source will be persisted as record. { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "name": "grp", + "out.dataframe.name": "grp", "rule": "select name, count(*) as `cnt` from source group by name", "out": [ { @@ -252,7 +249,9 @@ The miss records of source will be persisted as record. ] } ] - } + }, + + "sinks": ["CONSOLE","ELASTICSEARCH"] } ``` Above is the configure file of streaming profiling job. http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/3545a71c/measure/src/main/scala/org/apache/griffin/measure/configuration/dqdefinition/DQConfig.scala ---------------------------------------------------------------------- diff --git a/measure/src/main/scala/org/apache/griffin/measure/configuration/dqdefinition/DQConfig.scala b/measure/src/main/scala/org/apache/griffin/measure/configuration/dqdefinition/DQConfig.scala index a4cdfc1..d41abf3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/configuration/dqdefinition/DQConfig.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/configuration/dqdefinition/DQConfig.scala @@ -141,9 +141,6 @@ case class EvaluateRuleParam( @JsonProperty("rules") private val rules: List[Rul * @param details detail config of rule (optional) * @param cache cache the result for multiple usage (optional, valid for "spark-sql" and "df-ops" mode) * @param outputs output ways configuration (optional) -// * @param metric config for metric output (optional) -// * @param record config for record output (optional) -// * @param dsCacheUpdate config for data source cache update output (optional, valid in streaming mode) */ @JsonInclude(Include.NON_NULL) case class RuleParam(@JsonProperty("dsl.type") private val dslType: String,